|
- %{
- // 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 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.
-
- // Inital yacc source generated by ebnf2y[1]
- // at 2013-10-04 23:10:47.861401015 +0200 CEST
- //
- // $ ebnf2y -o ql.y -oe ql.ebnf -start StatementList -pkg ql -p _
- //
- // [1]: http://github.com/cznic/ebnf2y
-
- package parser
-
- import (
- "strings"
-
- "github.com/pingcap/tidb/mysql"
- "github.com/pingcap/tidb/ast"
- "github.com/pingcap/tidb/model"
- "github.com/pingcap/tidb/parser/opcode"
- "github.com/pingcap/tidb/util/charset"
- "github.com/pingcap/tidb/util/types"
- )
-
- %}
-
- %union {
- offset int // offset
- line int
- col int
- item interface{}
- list []interface{}
- }
-
- %token <item>
-
- /*yy:token "1.%d" */ floatLit "floating-point literal"
- /*yy:token "%c" */ identifier "identifier"
- /*yy:token "%d" */ intLit "integer literal"
- /*yy:token "\"%c\"" */ stringLit "string literal"
- /*yy:token "%x" */ hexLit "hexadecimal literal"
- /*yy:token "%b" */ bitLit "bit literal"
-
-
- abs "ABS"
- add "ADD"
- addDate "ADDDATE"
- admin "ADMIN"
- after "AFTER"
- all "ALL"
- alter "ALTER"
- and "AND"
- andand "&&"
- andnot "&^"
- any "ANY"
- as "AS"
- asc "ASC"
- at "AT"
- autoIncrement "AUTO_INCREMENT"
- avg "AVG"
- avgRowLength "AVG_ROW_LENGTH"
- begin "BEGIN"
- between "BETWEEN"
- both "BOTH"
- btree "BTREE"
- by "BY"
- byteType "BYTE"
- caseKwd "CASE"
- cast "CAST"
- character "CHARACTER"
- charsetKwd "CHARSET"
- check "CHECK"
- checksum "CHECKSUM"
- coalesce "COALESCE"
- collate "COLLATE"
- collation "COLLATION"
- column "COLUMN"
- columns "COLUMNS"
- comment "COMMENT"
- commit "COMMIT"
- committed "COMMITTED"
- compact "COMPACT"
- compressed "COMPRESSED"
- compression "COMPRESSION"
- concat "CONCAT"
- concatWs "CONCAT_WS"
- connection "CONNECTION"
- connectionID "CONNECTION_ID"
- constraint "CONSTRAINT"
- convert "CONVERT"
- count "COUNT"
- create "CREATE"
- cross "CROSS"
- curDate "CURDATE"
- currentDate "CURRENT_DATE"
- curTime "CUR_TIME"
- currentTime "CURRENT_TIME"
- currentUser "CURRENT_USER"
- database "DATABASE"
- databases "DATABASES"
- dateAdd "DATE_ADD"
- dateSub "DATE_SUB"
- day "DAY"
- dayname "DAYNAME"
- dayofmonth "DAYOFMONTH"
- dayofweek "DAYOFWEEK"
- dayofyear "DAYOFYEAR"
- ddl "DDL"
- deallocate "DEALLOCATE"
- defaultKwd "DEFAULT"
- delayed "DELAYED"
- delayKeyWrite "DELAY_KEY_WRITE"
- deleteKwd "DELETE"
- desc "DESC"
- describe "DESCRIBE"
- distinct "DISTINCT"
- div "DIV"
- do "DO"
- drop "DROP"
- dual "DUAL"
- duplicate "DUPLICATE"
- dynamic "DYNAMIC"
- elseKwd "ELSE"
- end "END"
- engine "ENGINE"
- engines "ENGINES"
- enum "ENUM"
- eq "="
- escape "ESCAPE"
- execute "EXECUTE"
- exists "EXISTS"
- explain "EXPLAIN"
- extract "EXTRACT"
- falseKwd "false"
- fields "FIELDS"
- first "FIRST"
- fixed "FIXED"
- foreign "FOREIGN"
- forKwd "FOR"
- foundRows "FOUND_ROWS"
- from "FROM"
- full "FULL"
- fulltext "FULLTEXT"
- ge ">="
- global "GLOBAL"
- grant "GRANT"
- grants "GRANTS"
- group "GROUP"
- groupConcat "GROUP_CONCAT"
- hash "HASH"
- having "HAVING"
- highPriority "HIGH_PRIORITY"
- hour "HOUR"
- identified "IDENTIFIED"
- ignore "IGNORE"
- ifKwd "IF"
- ifNull "IFNULL"
- in "IN"
- index "INDEX"
- inner "INNER"
- insert "INSERT"
- interval "INTERVAL"
- into "INTO"
- is "IS"
- isolation "ISOLATION"
- join "JOIN"
- key "KEY"
- keyBlockSize "KEY_BLOCK_SIZE"
- le "<="
- leading "LEADING"
- left "LEFT"
- length "LENGTH"
- level "LEVEL"
- like "LIKE"
- limit "LIMIT"
- local "LOCAL"
- locate "LOCATE"
- lock "LOCK"
- lower "LOWER"
- lowPriority "LOW_PRIORITY"
- lsh "<<"
- max "MAX"
- maxRows "MAX_ROWS"
- microsecond "MICROSECOND"
- min "MIN"
- minute "MINUTE"
- minRows "MIN_ROWS"
- mod "MOD"
- mode "MODE"
- month "MONTH"
- names "NAMES"
- national "NATIONAL"
- neq "!="
- neqSynonym "<>"
- not "NOT"
- null "NULL"
- nulleq "<=>"
- nullIf "NULLIF"
- offset "OFFSET"
- on "ON"
- only "ONLY"
- option "OPTION"
- or "OR"
- order "ORDER"
- oror "||"
- outer "OUTER"
- password "PASSWORD"
- placeholder "PLACEHOLDER"
- pow "POW"
- power "POWER"
- prepare "PREPARE"
- primary "PRIMARY"
- procedure "PROCEDURE"
- quarter "QUARTER"
- quick "QUICK"
- rand "RAND"
- read "READ"
- redundant "REDUNDANT"
- references "REFERENCES"
- regexpKwd "REGEXP"
- repeat "REPEAT"
- repeatable "REPEATABLE"
- replace "REPLACE"
- right "RIGHT"
- rlike "RLIKE"
- rollback "ROLLBACK"
- row "ROW"
- rowFormat "ROW_FORMAT"
- rsh ">>"
- schema "SCHEMA"
- schemas "SCHEMAS"
- second "SECOND"
- selectKwd "SELECT"
- serializable "SERIALIZABLE"
- session "SESSION"
- set "SET"
- share "SHARE"
- show "SHOW"
- signed "SIGNED"
- some "SOME"
- start "START"
- status "STATUS"
- stringType "string"
- subDate "SUBDATE"
- strcmp "STRCMP"
- substring "SUBSTRING"
- substringIndex "SUBSTRING_INDEX"
- sum "SUM"
- sysVar "SYS_VAR"
- sysDate "SYSDATE"
- tableKwd "TABLE"
- tables "TABLES"
- then "THEN"
- to "TO"
- trailing "TRAILING"
- transaction "TRANSACTION"
- triggers "TRIGGERS"
- trim "TRIM"
- trueKwd "true"
- truncate "TRUNCATE"
- uncommitted "UNCOMMITTED"
- underscoreCS "UNDERSCORE_CHARSET"
- unknown "UNKNOWN"
- union "UNION"
- unique "UNIQUE"
- unlock "UNLOCK"
- unsigned "UNSIGNED"
- update "UPDATE"
- upper "UPPER"
- use "USE"
- user "USER"
- using "USING"
- userVar "USER_VAR"
- value "VALUE"
- values "VALUES"
- variables "VARIABLES"
- version "VERSION"
- warnings "WARNINGS"
- week "WEEK"
- weekday "WEEKDAY"
- weekofyear "WEEKOFYEAR"
- when "WHEN"
- where "WHERE"
- write "WRITE"
- xor "XOR"
- yearweek "YEARWEEK"
- zerofill "ZEROFILL"
-
- calcFoundRows "SQL_CALC_FOUND_ROWS"
-
- currentTs "CURRENT_TIMESTAMP"
- localTime "LOCALTIME"
- localTs "LOCALTIMESTAMP"
- now "NOW"
-
- tinyIntType "TINYINT"
- smallIntType "SMALLINT"
- mediumIntType "MEDIUMINT"
- intType "INT"
- integerType "INTEGER"
- bigIntType "BIGINT"
- bitType "BIT"
-
- decimalType "DECIMAL"
- numericType "NUMERIC"
- floatType "float"
- doubleType "DOUBLE"
- precisionType "PRECISION"
- realType "REAL"
-
- dateType "DATE"
- timeType "TIME"
- datetimeType "DATETIME"
- timestampType "TIMESTAMP"
- yearType "YEAR"
-
- charType "CHAR"
- varcharType "VARCHAR"
- binaryType "BINARY"
- varbinaryType "VARBINARY"
- tinyblobType "TINYBLOB"
- blobType "BLOB"
- mediumblobType "MEDIUMBLOB"
- longblobType "LONGBLOB"
- tinytextType "TINYTEXT"
- textType "TEXT"
- mediumtextType "MEDIUMTEXT"
- longtextType "LONGTEXT"
-
- int16Type "int16"
- int24Type "int24"
- int32Type "int32"
- int64Type "int64"
- int8Type "int8"
- uintType "uint"
- uint16Type "uint16"
- uint32Type "uint32"
- uint64Type "uint64"
- uint8Type "uint8",
- float32Type "float32"
- float64Type "float64"
- boolType "BOOL"
- booleanType "BOOLEAN"
-
- parseExpression "parse expression prefix"
-
- secondMicrosecond "SECOND_MICROSECOND"
- minuteMicrosecond "MINUTE_MICROSECOND"
- minuteSecond "MINUTE_SECOND"
- hourMicrosecond "HOUR_MICROSECOND"
- hourSecond "HOUR_SECOND"
- hourMinute "HOUR_MINUTE"
- dayMicrosecond "DAY_MICROSECOND"
- daySecond "DAY_SECOND"
- dayMinute "DAY_MINUTE"
- dayHour "DAY_HOUR"
- yearMonth "YEAR_MONTH"
-
- %type <item>
- AdminStmt "Check table statement or show ddl statement"
- AlterTableStmt "Alter table statement"
- AlterTableSpec "Alter table specification"
- AlterTableSpecList "Alter table specification list"
- AnyOrAll "Any or All for subquery"
- Assignment "assignment"
- AssignmentList "assignment list"
- AssignmentListOpt "assignment list opt"
- AuthOption "User auth option"
- AuthString "Password string value"
- BeginTransactionStmt "BEGIN TRANSACTION statement"
- CastType "Cast function target type"
- ColumnDef "table column definition"
- ColumnName "column name"
- ColumnNameList "column name list"
- ColumnNameListOpt "column name list opt"
- ColumnKeywordOpt "Column keyword or empty"
- ColumnSetValue "insert statement set value by column name"
- ColumnSetValueList "insert statement set value by column name list"
- CommaOpt "optional comma"
- CommitStmt "COMMIT statement"
- CompareOp "Compare opcode"
- ColumnOption "column definition option"
- ColumnOptionList "column definition option list"
- ColumnOptionListOpt "optional column definition option list"
- Constraint "table constraint"
- ConstraintElem "table constraint element"
- ConstraintKeywordOpt "Constraint Keyword or empty"
- CreateDatabaseStmt "Create Database Statement"
- CreateIndexStmt "CREATE INDEX statement"
- CreateIndexStmtUnique "CREATE INDEX optional UNIQUE clause"
- DatabaseOption "CREATE Database specification"
- DatabaseOptionList "CREATE Database specification list"
- DatabaseOptionListOpt "CREATE Database specification list opt"
- CreateTableStmt "CREATE TABLE statement"
- CreateUserStmt "CREATE User statement"
- CrossOpt "Cross join option"
- DateArithOpt "Date arith dateadd or datesub option"
- DateArithMultiFormsOpt "Date arith adddate or subdate option"
- DateArithInterval "Date arith interval part"
- DatabaseSym "DATABASE or SCHEMA"
- DBName "Database Name"
- DeallocateSym "Deallocate or drop"
- DeallocateStmt "Deallocate prepared statement"
- Default "DEFAULT clause"
- DefaultOpt "optional DEFAULT clause"
- DefaultKwdOpt "optional DEFAULT keyword"
- DefaultValueExpr "DefaultValueExpr(Now or Signed Literal)"
- DeleteFromStmt "DELETE FROM statement"
- DistinctOpt "Distinct option"
- DoStmt "Do statement"
- DropDatabaseStmt "DROP DATABASE statement"
- DropIndexStmt "DROP INDEX statement"
- DropTableStmt "DROP TABLE statement"
- EmptyStmt "empty statement"
- EqOpt "= or empty"
- EscapedTableRef "escaped table reference"
- ExecuteStmt "Execute statement"
- ExplainSym "EXPLAIN or DESCRIBE or DESC"
- ExplainStmt "EXPLAIN statement"
- Expression "expression"
- ExpressionList "expression list"
- ExpressionListOpt "expression list opt"
- ExpressionListList "expression list list"
- Factor "expression factor"
- PredicateExpr "Predicate expression factor"
- Field "field expression"
- FieldAsName "Field alias name"
- FieldAsNameOpt "Field alias name opt"
- FieldList "field expression list"
- TableRefsClause "Table references clause"
- Function "function expr"
- FunctionCallAgg "Function call on aggregate data"
- FunctionCallConflict "Function call with reserved keyword as function name"
- FunctionCallKeyword "Function call with keyword as function name"
- FunctionCallNonKeyword "Function call with nonkeyword as function name"
- FunctionNameConflict "Built-in function call names which are conflict with keywords"
- FuncDatetimePrec "Function datetime precision"
- GlobalScope "The scope of variable"
- GrantStmt "Grant statement"
- GroupByClause "GROUP BY clause"
- HashString "Hashed string"
- HavingClause "HAVING clause"
- IfExists "If Exists"
- IfNotExists "If Not Exists"
- IgnoreOptional "IGNORE or empty"
- IndexColName "Index column name"
- IndexColNameList "List of index column name"
- IndexName "index name"
- IndexOption "Index Option"
- IndexType "index type"
- IndexTypeOpt "Optional index type"
- InsertIntoStmt "INSERT INTO statement"
- InsertValues "Rest part of INSERT/REPLACE INTO statement"
- IntoOpt "INTO or EmptyString"
- IsolationLevel "Isolation level"
- JoinTable "join table"
- JoinType "join type"
- KeyOrIndex "{KEY|INDEX}"
- LikeEscapeOpt "like escape option"
- LimitClause "LIMIT clause"
- Literal "literal value"
- LockTablesStmt "Lock tables statement"
- LockType "Table locks type"
- logAnd "logical and operator"
- logOr "logical or operator"
- LowPriorityOptional "LOW_PRIORITY or empty"
- name "name"
- NationalOpt "National option"
- NotOpt "optional NOT"
- NowSym "CURRENT_TIMESTAMP/LOCALTIME/LOCALTIMESTAMP/NOW"
- NumLiteral "Num/Int/Float/Decimal Literal"
- ObjectType "Grant statement object type"
- OnDuplicateKeyUpdate "ON DUPLICATE KEY UPDATE value list"
- Operand "operand"
- OptFull "Full or empty"
- OptInteger "Optional Integer keyword"
- Order "ORDER BY clause optional collation specification"
- OrderBy "ORDER BY clause"
- ByItem "BY item"
- OrderByOptional "Optional ORDER BY clause optional"
- ByList "BY list"
- OuterOpt "optional OUTER clause"
- QuickOptional "QUICK or empty"
- PasswordOpt "Password option"
- ColumnPosition "Column position [First|After ColumnName]"
- PreparedStmt "PreparedStmt"
- PrepareSQL "Prepare statement sql string"
- PrimaryExpression "primary expression"
- PrimaryFactor "primary expression factor"
- Priority "insert statement priority"
- PrivElem "Privilege element"
- PrivElemList "Privilege element list"
- PrivLevel "Privilege scope"
- PrivType "Privilege type"
- ReferDef "Reference definition"
- RegexpSym "REGEXP or RLIKE"
- ReplaceIntoStmt "REPLACE INTO statement"
- ReplacePriority "replace statement priority"
- RollbackStmt "ROLLBACK statement"
- RowFormat "Row format option"
- SelectLockOpt "FOR UPDATE or LOCK IN SHARE MODE,"
- SelectStmt "SELECT statement"
- SelectStmtCalcFoundRows "SELECT statement optional SQL_CALC_FOUND_ROWS"
- SelectStmtDistinct "SELECT statement optional DISTINCT clause"
- SelectStmtFieldList "SELECT statement field list"
- SelectStmtLimit "SELECT statement optional LIMIT clause"
- SelectStmtOpts "Select statement options"
- SelectStmtGroup "SELECT statement optional GROUP BY clause"
- SetStmt "Set variable statement"
- ShowStmt "Show engines/databases/tables/columns/warnings/status statement"
- ShowTargetFilterable "Show target that can be filtered by WHERE or LIKE"
- ShowDatabaseNameOpt "Show tables/columns statement database name option"
- ShowTableAliasOpt "Show table alias option"
- ShowLikeOrWhereOpt "Show like or where clause option"
- SignedLiteral "Literal or NumLiteral with sign"
- Statement "statement"
- StatementList "statement list"
- StringName "string literal or identifier"
- StringList "string list"
- ExplainableStmt "explainable statement"
- SubSelect "Sub Select"
- Symbol "Constraint Symbol"
- SystemVariable "System defined variable name"
- TableAsName "table alias name"
- TableAsNameOpt "table alias name optional"
- TableElement "table definition element"
- TableElementList "table definition element list"
- TableFactor "table factor"
- TableLock "Table name and lock type"
- TableLockList "Table lock list"
- TableName "Table name"
- TableNameList "Table name list"
- TableOption "create table option"
- TableOptionList "create table option list"
- TableOptionListOpt "create table option list opt"
- TableRef "table reference"
- TableRefs "table references"
- TimeUnit "Time unit"
- TransactionChar "Transaction characteristic"
- TransactionChars "Transaction characteristic list"
- TrimDirection "Trim string direction"
- TruncateTableStmt "TRANSACTION TABLE statement"
- UnionOpt "Union Option(empty/ALL/DISTINCT)"
- UnionStmt "Union select state ment"
- UnionClauseList "Union select clause list"
- UnionSelect "Union (select) item"
- UnlockTablesStmt "Unlock tables statement"
- UpdateStmt "UPDATE statement"
- Username "Username"
- UserSpec "Username and auth option"
- UserSpecList "Username and auth option list"
- UserVariable "User defined variable name"
- UserVariableList "User defined variable name list"
- UseStmt "USE statement"
- ValueSym "Value or Values"
- VariableAssignment "set variable value"
- VariableAssignmentList "set variable value list"
- Variable "User or system variable"
- WhereClause "WHERE clause"
- WhereClauseOptional "Optinal WHERE clause"
-
- Identifier "identifier or unreserved keyword"
- UnReservedKeyword "MySQL unreserved keywords"
- NotKeywordToken "Tokens not mysql keyword but treated specially"
-
- WhenClause "When clause"
- WhenClauseList "When clause list"
- ElseOpt "Optional else clause"
- ExpressionOpt "Optional expression"
-
- Type "Types"
-
- NumericType "Numeric types"
- IntegerType "Integer Types types"
- FixedPointType "Exact value types"
- FloatingPointType "Approximate value types"
- BitValueType "bit value types"
-
- StringType "String types"
- BlobType "Blob types"
- TextType "Text types"
-
- DateAndTimeType "Date and Time types"
-
- OptFieldLen "Field length or empty"
- FieldLen "Field length"
- FieldOpts "Field type definition option list"
- FieldOpt "Field type definition option"
- FloatOpt "Floating-point type option"
- Precision "Floating-point precision option"
- OptBinary "Optional BINARY"
- CharsetKw "charset or charater set"
- OptCharset "Optional Character setting"
- OptCollate "Optional Collate setting"
- NUM "numbers"
- LengthNum "Field length num(uint64)"
-
- %token tableRefPriority
-
- %precedence lowerThanCalcFoundRows
- %precedence calcFoundRows
-
- %precedence lowerThanSetKeyword
- %precedence set
-
- %precedence lowerThanInsertValues
- %precedence insertValues
-
- %left join inner cross left right full
- /* A dummy token to force the priority of TableRef production in a join. */
- %left tableRefPriority
- %precedence on
- %left oror or
- %left xor
- %left andand and
- %left between
- %precedence lowerThanEq
- %left eq ge le neq neqSynonym '>' '<' is like in
- %left '|'
- %left '&'
- %left rsh lsh
- %left '-' '+'
- %left '*' '/' '%' div mod
- %left '^'
- %left '~' neg
- %right not
- %right collate
-
- %precedence lowerThanLeftParen
- %precedence '('
- %precedence lowerThanQuick
- %precedence quick
- %precedence lowerThanEscape
- %precedence escape
- %precedence lowerThanComma
- %precedence ','
-
- %start Start
-
- %%
-
- Start:
- StatementList
- | parseExpression Expression
- {
- yylex.(*lexer).expr = $2.(ast.ExprNode)
- }
-
- /**************************************AlterTableStmt***************************************
- * See: https://dev.mysql.com/doc/refman/5.7/en/alter-table.html
- *******************************************************************************************/
- AlterTableStmt:
- "ALTER" IgnoreOptional "TABLE" TableName AlterTableSpecList
- {
- $$ = &ast.AlterTableStmt{
- Table: $4.(*ast.TableName),
- Specs: $5.([]*ast.AlterTableSpec),
- }
- }
-
- AlterTableSpec:
- TableOptionListOpt
- {
- $$ = &ast.AlterTableSpec{
- Tp: ast.AlterTableOption,
- Options:$1.([]*ast.TableOption),
- }
- }
- | "ADD" ColumnKeywordOpt ColumnDef ColumnPosition
- {
- $$ = &ast.AlterTableSpec{
- Tp: ast.AlterTableAddColumn,
- Column: $3.(*ast.ColumnDef),
- Position: $4.(*ast.ColumnPosition),
- }
- }
- | "ADD" Constraint
- {
- constraint := $2.(*ast.Constraint)
- $$ = &ast.AlterTableSpec{
- Tp: ast.AlterTableAddConstraint,
- Constraint: constraint,
- }
- }
- | "DROP" ColumnKeywordOpt ColumnName
- {
- $$ = &ast.AlterTableSpec{
- Tp: ast.AlterTableDropColumn,
- DropColumn: $3.(*ast.ColumnName),
- }
- }
- | "DROP" "PRIMARY" "KEY"
- {
- $$ = &ast.AlterTableSpec{Tp: ast.AlterTableDropPrimaryKey}
- }
- | "DROP" KeyOrIndex IndexName
- {
- $$ = &ast.AlterTableSpec{
- Tp: ast.AlterTableDropIndex,
- Name: $3.(string),
- }
- }
- | "DROP" "FOREIGN" "KEY" Symbol
- {
- $$ = &ast.AlterTableSpec{
- Tp: ast.AlterTableDropForeignKey,
- Name: $4.(string),
- }
- }
-
- KeyOrIndex:
- "KEY"|"INDEX"
-
- ColumnKeywordOpt:
- {}
- | "COLUMN"
-
- ColumnPosition:
- {
- $$ = &ast.ColumnPosition{Tp: ast.ColumnPositionNone}
- }
- | "FIRST"
- {
- $$ = &ast.ColumnPosition{Tp: ast.ColumnPositionFirst}
- }
- | "AFTER" ColumnName
- {
- $$ = &ast.ColumnPosition{
- Tp: ast.ColumnPositionAfter,
- RelativeColumn: $2.(*ast.ColumnName),
- }
- }
-
- AlterTableSpecList:
- AlterTableSpec
- {
- $$ = []*ast.AlterTableSpec{$1.(*ast.AlterTableSpec)}
- }
- | AlterTableSpecList ',' AlterTableSpec
- {
- $$ = append($1.([]*ast.AlterTableSpec), $3.(*ast.AlterTableSpec))
- }
-
- ConstraintKeywordOpt:
- {
- $$ = nil
- }
- | "CONSTRAINT"
- {
- $$ = nil
- }
- | "CONSTRAINT" Symbol
- {
- $$ = $2.(string)
- }
-
- Symbol:
- Identifier
-
- /*******************************************************************************************/
- Assignment:
- ColumnName eq Expression
- {
- $$ = &ast.Assignment{Column: $1.(*ast.ColumnName), Expr:$3.(ast.ExprNode)}
- }
-
- AssignmentList:
- Assignment
- {
- $$ = []*ast.Assignment{$1.(*ast.Assignment)}
- }
- | AssignmentList ',' Assignment
- {
- $$ = append($1.([]*ast.Assignment), $3.(*ast.Assignment))
- }
-
- AssignmentListOpt:
- /* EMPTY */
- {
- $$ = []*ast.Assignment{}
- }
- | AssignmentList
-
- BeginTransactionStmt:
- "BEGIN"
- {
- $$ = &ast.BeginStmt{}
- }
- | "START" "TRANSACTION"
- {
- $$ = &ast.BeginStmt{}
- }
-
- ColumnDef:
- ColumnName Type ColumnOptionListOpt
- {
- $$ = &ast.ColumnDef{Name: $1.(*ast.ColumnName), Tp: $2.(*types.FieldType), Options: $3.([]*ast.ColumnOption)}
- }
-
- ColumnName:
- Identifier
- {
- $$ = &ast.ColumnName{Name: model.NewCIStr($1.(string))}
- }
- | Identifier '.' Identifier
- {
- $$ = &ast.ColumnName{Table: model.NewCIStr($1.(string)), Name: model.NewCIStr($3.(string))}
- }
- | Identifier '.' Identifier '.' Identifier
- {
- $$ = &ast.ColumnName{Schema: model.NewCIStr($1.(string)), Table: model.NewCIStr($3.(string)), Name: model.NewCIStr($5.(string))}
- }
-
- ColumnNameList:
- ColumnName
- {
- $$ = []*ast.ColumnName{$1.(*ast.ColumnName)}
- }
- | ColumnNameList ',' ColumnName
- {
- $$ = append($1.([]*ast.ColumnName), $3.(*ast.ColumnName))
- }
-
- ColumnNameListOpt:
- /* EMPTY */
- {
- $$ = []*ast.ColumnName{}
- }
- | ColumnNameList
- {
- $$ = $1.([]*ast.ColumnName)
- }
-
- CommitStmt:
- "COMMIT"
- {
- $$ = &ast.CommitStmt{}
- }
-
- ColumnOption:
- "NOT" "NULL"
- {
- $$ = &ast.ColumnOption{Tp: ast.ColumnOptionNotNull}
- }
- | "NULL"
- {
- $$ = &ast.ColumnOption{Tp: ast.ColumnOptionNull}
- }
- | "AUTO_INCREMENT"
- {
- $$ = &ast.ColumnOption{Tp: ast.ColumnOptionAutoIncrement}
- }
- | "PRIMARY" "KEY"
- {
- $$ = &ast.ColumnOption{Tp: ast.ColumnOptionPrimaryKey}
- }
- | "UNIQUE"
- {
- $$ = &ast.ColumnOption{Tp: ast.ColumnOptionUniq}
- }
- | "UNIQUE" "KEY"
- {
- $$ = &ast.ColumnOption{Tp: ast.ColumnOptionUniqKey}
- }
- | "DEFAULT" DefaultValueExpr
- {
- $$ = &ast.ColumnOption{Tp: ast.ColumnOptionDefaultValue, Expr: $2.(ast.ExprNode)}
- }
- | "ON" "UPDATE" NowSym
- {
- nowFunc := &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP")}
- $$ = &ast.ColumnOption{Tp: ast.ColumnOptionOnUpdate, Expr: nowFunc}
- }
- | "COMMENT" stringLit
- {
- $$ = &ast.ColumnOption{Tp: ast.ColumnOptionComment}
- }
- | "CHECK" '(' Expression ')'
- {
- // See: https://dev.mysql.com/doc/refman/5.7/en/create-table.html
- // The CHECK clause is parsed but ignored by all storage engines.
- $$ = &ast.ColumnOption{}
- }
-
- ColumnOptionList:
- ColumnOption
- {
- $$ = []*ast.ColumnOption{$1.(*ast.ColumnOption)}
- }
- | ColumnOptionList ColumnOption
- {
- $$ = append($1.([]*ast.ColumnOption), $2.(*ast.ColumnOption))
- }
-
- ColumnOptionListOpt:
- {
- $$ = []*ast.ColumnOption{}
- }
- | ColumnOptionList
- {
- $$ = $1.([]*ast.ColumnOption)
- }
-
- ConstraintElem:
- "PRIMARY" "KEY" IndexTypeOpt '(' IndexColNameList ')' IndexOption
- {
- c := &ast.Constraint{
- Tp: ast.ConstraintPrimaryKey,
- Keys: $5.([]*ast.IndexColName),
- }
- if $7 != nil {
- c.Option = $7.(*ast.IndexOption)
- }
- if $3 != nil {
- if c.Option == nil {
- c.Option = &ast.IndexOption{}
- }
- c.Option.Tp = $3.(model.IndexType)
- }
- $$ = c
- }
- | "FULLTEXT" "KEY" IndexName '(' IndexColNameList ')' IndexOption
- {
- c := &ast.Constraint{
- Tp: ast.ConstraintFulltext,
- Keys: $5.([]*ast.IndexColName),
- Name: $3.(string),
- }
- if $7 != nil {
- c.Option = $7.(*ast.IndexOption)
- }
- $$ = c
- }
- | "INDEX" IndexName IndexTypeOpt '(' IndexColNameList ')' IndexOption
- {
- c := &ast.Constraint{
- Tp: ast.ConstraintIndex,
- Keys: $5.([]*ast.IndexColName),
- Name: $2.(string),
- }
- if $7 != nil {
- c.Option = $7.(*ast.IndexOption)
- }
- if $3 != nil {
- if c.Option == nil {
- c.Option = &ast.IndexOption{}
- }
- c.Option.Tp = $3.(model.IndexType)
- }
- $$ = c
- }
- | "KEY" IndexName IndexTypeOpt '(' IndexColNameList ')' IndexOption
- {
- c := &ast.Constraint{
- Tp: ast.ConstraintKey,
- Keys: $5.([]*ast.IndexColName),
- Name: $2.(string),
- }
- if $7 != nil {
- c.Option = $7.(*ast.IndexOption)
- }
- if $3 != nil {
- if c.Option == nil {
- c.Option = &ast.IndexOption{}
- }
- c.Option.Tp = $3.(model.IndexType)
- }
- $$ = c
- }
- | "UNIQUE" IndexName IndexTypeOpt '(' IndexColNameList ')' IndexOption
- {
- c := &ast.Constraint{
- Tp: ast.ConstraintUniq,
- Keys: $5.([]*ast.IndexColName),
- Name: $2.(string),
- }
- if $7 != nil {
- c.Option = $7.(*ast.IndexOption)
- }
- if $3 != nil {
- if c.Option == nil {
- c.Option = &ast.IndexOption{}
- }
- c.Option.Tp = $3.(model.IndexType)
- }
- $$ = c
- }
- | "UNIQUE" "INDEX" IndexName IndexTypeOpt '(' IndexColNameList ')' IndexOption
- {
- c := &ast.Constraint{
- Tp: ast.ConstraintUniqIndex,
- Keys: $6.([]*ast.IndexColName),
- Name: $3.(string),
- }
- if $8 != nil {
- c.Option = $8.(*ast.IndexOption)
- }
- if $4 != nil {
- if c.Option == nil {
- c.Option = &ast.IndexOption{}
- }
- c.Option.Tp = $4.(model.IndexType)
- }
- $$ = c
- }
- | "UNIQUE" "KEY" IndexName IndexTypeOpt '(' IndexColNameList ')' IndexOption
- {
- c := &ast.Constraint{
- Tp: ast.ConstraintUniqKey,
- Keys: $6.([]*ast.IndexColName),
- Name: $3.(string),
- }
- if $8 != nil {
- c.Option = $8.(*ast.IndexOption)
- }
- if $4 != nil {
- if c.Option == nil {
- c.Option = &ast.IndexOption{}
- }
- c.Option.Tp = $4.(model.IndexType)
- }
- $$ = c
- }
- | "FOREIGN" "KEY" IndexName '(' IndexColNameList ')' ReferDef
- {
- $$ = &ast.Constraint{
- Tp: ast.ConstraintForeignKey,
- Keys: $5.([]*ast.IndexColName),
- Name: $3.(string),
- Refer: $7.(*ast.ReferenceDef),
- }
- }
-
- ReferDef:
- "REFERENCES" TableName '(' IndexColNameList ')'
- {
- $$ = &ast.ReferenceDef{Table: $2.(*ast.TableName), IndexColNames: $4.([]*ast.IndexColName)}
- }
-
- /*
- * The DEFAULT clause specifies a default value for a column.
- * With one exception, the default value must be a constant;
- * it cannot be a function or an expression. This means, for example,
- * that you cannot set the default for a date column to be the value of
- * a function such as NOW() or CURRENT_DATE. The exception is that you
- * can specify CURRENT_TIMESTAMP as the default for a TIMESTAMP or DATETIME column.
- *
- * See: http://dev.mysql.com/doc/refman/5.7/en/create-table.html
- * https://github.com/mysql/mysql-server/blob/5.7/sql/sql_yacc.yy#L6832
- */
- DefaultValueExpr:
- NowSym
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP")}
- }
- | NowSym '(' ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr("CURRENT_TIMESTAMP")}
- }
- | SignedLiteral
-
- // TODO: Process other three keywords
- NowSym:
- "CURRENT_TIMESTAMP"
- | "LOCALTIME"
- | "LOCALTIMESTAMP"
- | "NOW"
-
- SignedLiteral:
- Literal
- {
- $$ = ast.NewValueExpr($1)
- }
- | '+' NumLiteral
- {
- $$ = &ast.UnaryOperationExpr{Op: opcode.Plus, V: ast.NewValueExpr($2)}
- }
- | '-' NumLiteral
- {
- $$ = &ast.UnaryOperationExpr{Op: opcode.Minus, V: ast.NewValueExpr($2)}
- }
-
- // TODO: support decimal literal
- NumLiteral:
- intLit
- | floatLit
-
-
- CreateIndexStmt:
- "CREATE" CreateIndexStmtUnique "INDEX" Identifier "ON" TableName '(' IndexColNameList ')'
- {
- $$ = &ast.CreateIndexStmt{
- Unique: $2.(bool),
- IndexName: $4.(string),
- Table: $6.(*ast.TableName),
- IndexColNames: $8.([]*ast.IndexColName),
- }
- if yylex.(*lexer).root {
- break
- }
- }
-
- CreateIndexStmtUnique:
- {
- $$ = false
- }
- | "UNIQUE"
- {
- $$ = true
- }
-
- IndexColName:
- ColumnName OptFieldLen Order
- {
- //Order is parsed but just ignored as MySQL did
- $$ = &ast.IndexColName{Column: $1.(*ast.ColumnName), Length: $2.(int)}
- }
-
- IndexColNameList:
- {
- $$ = []*ast.IndexColName{}
- }
- | IndexColName
- {
- $$ = []*ast.IndexColName{$1.(*ast.IndexColName)}
- }
- | IndexColNameList ',' IndexColName
- {
- $$ = append($1.([]*ast.IndexColName), $3.(*ast.IndexColName))
- }
-
-
-
- /*******************************************************************
- *
- * Create Database Statement
- * CREATE {DATABASE | SCHEMA} [IF NOT EXISTS] db_name
- * [create_specification] ...
- *
- * create_specification:
- * [DEFAULT] CHARACTER SET [=] charset_name
- * | [DEFAULT] COLLATE [=] collation_name
- *******************************************************************/
- CreateDatabaseStmt:
- "CREATE" DatabaseSym IfNotExists DBName DatabaseOptionListOpt
- {
- $$ = &ast.CreateDatabaseStmt{
- IfNotExists: $3.(bool),
- Name: $4.(string),
- Options: $5.([]*ast.DatabaseOption),
- }
-
- if yylex.(*lexer).root {
- break
- }
- }
-
- DBName:
- Identifier
-
- DatabaseOption:
- DefaultKwdOpt CharsetKw EqOpt StringName
- {
- $$ = &ast.DatabaseOption{Tp: ast.DatabaseOptionCharset, Value: $4.(string)}
- }
- | DefaultKwdOpt "COLLATE" EqOpt StringName
- {
- $$ = &ast.DatabaseOption{Tp: ast.DatabaseOptionCollate, Value: $4.(string)}
- }
-
- DatabaseOptionListOpt:
- {
- $$ = []*ast.DatabaseOption{}
- }
- | DatabaseOptionList
-
- DatabaseOptionList:
- DatabaseOption
- {
- $$ = []*ast.DatabaseOption{$1.(*ast.DatabaseOption)}
- }
- | DatabaseOptionList DatabaseOption
- {
- $$ = append($1.([]*ast.DatabaseOption), $2.(*ast.DatabaseOption))
- }
-
- /*******************************************************************
- *
- * Create Table Statement
- *
- * Example:
- * CREATE TABLE Persons
- * (
- * P_Id int NOT NULL,
- * LastName varchar(255) NOT NULL,
- * FirstName varchar(255),
- * Address varchar(255),
- * City varchar(255),
- * PRIMARY KEY (P_Id)
- * )
- *******************************************************************/
- CreateTableStmt:
- "CREATE" "TABLE" IfNotExists TableName '(' TableElementList ')' TableOptionListOpt
- {
- tes := $6.([]interface {})
- var columnDefs []*ast.ColumnDef
- var constraints []*ast.Constraint
- for _, te := range tes {
- switch te := te.(type) {
- case *ast.ColumnDef:
- columnDefs = append(columnDefs, te)
- case *ast.Constraint:
- constraints = append(constraints, te)
- }
- }
- if len(columnDefs) == 0 {
- yylex.(*lexer).err("Column Definition List can't be empty.")
- return 1
- }
- $$ = &ast.CreateTableStmt{
- Table: $4.(*ast.TableName),
- IfNotExists: $3.(bool),
- Cols: columnDefs,
- Constraints: constraints,
- Options: $8.([]*ast.TableOption),
- }
- }
-
- Default:
- "DEFAULT" Expression
- {
- $$ = $2
- }
-
- DefaultOpt:
- {
- $$ = nil
- }
- | Default
-
- DefaultKwdOpt:
- {}
- | "DEFAULT"
-
- /******************************************************************
- * Do statement
- * See: https://dev.mysql.com/doc/refman/5.7/en/do.html
- ******************************************************************/
- DoStmt:
- "DO" ExpressionList
- {
- $$ = &ast.DoStmt {
- Exprs: $2.([]ast.ExprNode),
- }
- }
-
- /*******************************************************************
- *
- * Delete Statement
- *
- *******************************************************************/
- DeleteFromStmt:
- "DELETE" LowPriorityOptional QuickOptional IgnoreOptional "FROM" TableName WhereClauseOptional OrderByOptional LimitClause
- {
- // Single Table
- join := &ast.Join{Left: &ast.TableSource{Source: $6.(ast.ResultSetNode)}, Right: nil}
- x := &ast.DeleteStmt{
- TableRefs: &ast.TableRefsClause{TableRefs: join},
- LowPriority: $2.(bool),
- Quick: $3.(bool),
- Ignore: $4.(bool),
- }
- if $7 != nil {
- x.Where = $7.(ast.ExprNode)
- }
- if $8 != nil {
- x.Order = $8.(*ast.OrderByClause)
- }
- if $9 != nil {
- x.Limit = $9.(*ast.Limit)
- }
-
- $$ = x
- if yylex.(*lexer).root {
- break
- }
- }
- | "DELETE" LowPriorityOptional QuickOptional IgnoreOptional TableNameList "FROM" TableRefs WhereClauseOptional
- {
- // Multiple Table
- x := &ast.DeleteStmt{
- LowPriority: $2.(bool),
- Quick: $3.(bool),
- Ignore: $4.(bool),
- IsMultiTable: true,
- BeforeFrom: true,
- Tables: &ast.DeleteTableList{Tables: $5.([]*ast.TableName)},
- TableRefs: &ast.TableRefsClause{TableRefs: $7.(*ast.Join)},
- }
- if $8 != nil {
- x.Where = $8.(ast.ExprNode)
- }
- $$ = x
- if yylex.(*lexer).root {
- break
- }
- }
- | "DELETE" LowPriorityOptional QuickOptional IgnoreOptional "FROM" TableNameList "USING" TableRefs WhereClauseOptional
- {
- // Multiple Table
- x := &ast.DeleteStmt{
- LowPriority: $2.(bool),
- Quick: $3.(bool),
- Ignore: $4.(bool),
- IsMultiTable: true,
- Tables: &ast.DeleteTableList{Tables: $6.([]*ast.TableName)},
- TableRefs: &ast.TableRefsClause{TableRefs: $8.(*ast.Join)},
- }
- if $9 != nil {
- x.Where = $9.(ast.ExprNode)
- }
- $$ = x
- if yylex.(*lexer).root {
- break
- }
- }
-
- DatabaseSym:
- "DATABASE" | "SCHEMA"
-
- DropDatabaseStmt:
- "DROP" DatabaseSym IfExists DBName
- {
- $$ = &ast.DropDatabaseStmt{IfExists: $3.(bool), Name: $4.(string)}
- if yylex.(*lexer).root {
- break
- }
- }
-
- DropIndexStmt:
- "DROP" "INDEX" IfExists Identifier "ON" TableName
- {
- $$ = &ast.DropIndexStmt{IfExists: $3.(bool), IndexName: $4.(string), Table: $6.(*ast.TableName)}
- }
-
- DropTableStmt:
- "DROP" TableOrTables TableNameList
- {
- $$ = &ast.DropTableStmt{Tables: $3.([]*ast.TableName)}
- if yylex.(*lexer).root {
- break
- }
- }
- | "DROP" TableOrTables "IF" "EXISTS" TableNameList
- {
- $$ = &ast.DropTableStmt{IfExists: true, Tables: $5.([]*ast.TableName)}
- if yylex.(*lexer).root {
- break
- }
- }
-
- TableOrTables:
- "TABLE"
- | "TABLES"
-
- EqOpt:
- {
- }
- | eq
- {
- }
-
- EmptyStmt:
- /* EMPTY */
- {
- $$ = nil
- }
-
- ExplainSym:
- "EXPLAIN"
- | "DESCRIBE"
- | "DESC"
-
- ExplainStmt:
- ExplainSym TableName
- {
- $$ = &ast.ExplainStmt{
- Stmt: &ast.ShowStmt{
- Tp: ast.ShowColumns,
- Table: $2.(*ast.TableName),
- },
- }
- }
- | ExplainSym TableName ColumnName
- {
- $$ = &ast.ExplainStmt{
- Stmt: &ast.ShowStmt{
- Tp: ast.ShowColumns,
- Table: $2.(*ast.TableName),
- Column: $3.(*ast.ColumnName),
- },
- }
- }
- | ExplainSym ExplainableStmt
- {
- $$ = &ast.ExplainStmt{Stmt: $2.(ast.StmtNode)}
- }
-
- LengthNum:
- NUM
- {
- switch v := $1.(type) {
- case int64:
- $$ = uint64(v)
- case uint64:
- $$ = uint64(v)
- }
- }
-
- NUM:
- intLit
-
- Expression:
- Expression logOr Expression %prec oror
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.OrOr, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | Expression "XOR" Expression %prec xor
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.LogicXor, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | Expression logAnd Expression %prec andand
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.AndAnd, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | "NOT" Expression %prec not
- {
- $$ = &ast.UnaryOperationExpr{Op: opcode.Not, V: $2.(ast.ExprNode)}
- }
- | Factor "IS" NotOpt trueKwd %prec is
- {
- $$ = &ast.IsTruthExpr{Expr:$1.(ast.ExprNode), Not: $3.(bool), True: int64(1)}
- }
- | Factor "IS" NotOpt falseKwd %prec is
- {
- $$ = &ast.IsTruthExpr{Expr:$1.(ast.ExprNode), Not: $3.(bool), True: int64(0)}
- }
- | Factor "IS" NotOpt "UNKNOWN" %prec is
- {
- /* https://dev.mysql.com/doc/refman/5.7/en/comparison-operators.html#operator_is */
- $$ = &ast.IsNullExpr{Expr: $1.(ast.ExprNode), Not: $3.(bool)}
- }
- | Factor
-
-
- logOr:
- "||"
- {
- }
- | "OR"
- {
- }
-
- logAnd:
- "&&"
- {
- }
- | "AND"
- {
- }
-
- name:
- Identifier
-
- ExpressionList:
- Expression
- {
- $$ = []ast.ExprNode{$1.(ast.ExprNode)}
- }
- | ExpressionList ',' Expression
- {
- $$ = append($1.([]ast.ExprNode), $3.(ast.ExprNode))
- }
-
- ExpressionListOpt:
- {
- $$ = []ast.ExprNode{}
- }
- | ExpressionList
-
- Factor:
- Factor "IS" NotOpt "NULL" %prec is
- {
- $$ = &ast.IsNullExpr{Expr: $1.(ast.ExprNode), Not: $3.(bool)}
- }
- | Factor CompareOp PredicateExpr %prec eq
- {
- $$ = &ast.BinaryOperationExpr{Op: $2.(opcode.Op), L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | Factor CompareOp AnyOrAll SubSelect %prec eq
- {
- $$ = &ast.CompareSubqueryExpr{Op: $2.(opcode.Op), L: $1.(ast.ExprNode), R: $4.(*ast.SubqueryExpr), All: $3.(bool)}
- }
- | PredicateExpr
-
- CompareOp:
- ">="
- {
- $$ = opcode.GE
- }
- | '>'
- {
- $$ = opcode.GT
- }
- | "<="
- {
- $$ = opcode.LE
- }
- | '<'
- {
- $$ = opcode.LT
- }
- | "!="
- {
- $$ = opcode.NE
- }
- | "<>"
- {
- $$ = opcode.NE
- }
- | "="
- {
- $$ = opcode.EQ
- }
- | "<=>"
- {
- $$ = opcode.NullEQ
- }
-
- AnyOrAll:
- "ANY"
- {
- $$ = false
- }
- | "SOME"
- {
- $$ = false
- }
- | "ALL"
- {
- $$ = true
- }
-
- PredicateExpr:
- PrimaryFactor NotOpt "IN" '(' ExpressionList ')'
- {
- $$ = &ast.PatternInExpr{Expr: $1.(ast.ExprNode), Not: $2.(bool), List: $5.([]ast.ExprNode)}
- }
- | PrimaryFactor NotOpt "IN" SubSelect
- {
- $$ = &ast.PatternInExpr{Expr: $1.(ast.ExprNode), Not: $2.(bool), Sel: $4.(*ast.SubqueryExpr)}
- }
- | PrimaryFactor NotOpt "BETWEEN" PrimaryFactor "AND" PredicateExpr
- {
- $$ = &ast.BetweenExpr{
- Expr: $1.(ast.ExprNode),
- Left: $4.(ast.ExprNode),
- Right: $6.(ast.ExprNode),
- Not: $2.(bool),
- }
- }
- | PrimaryFactor NotOpt "LIKE" PrimaryExpression LikeEscapeOpt
- {
- escape := $5.(string)
- if len(escape) > 1 {
- yylex.(*lexer).errf("Incorrect arguments %s to ESCAPE", escape)
- return 1
- } else if len(escape) == 0 {
- escape = "\\"
- }
- $$ = &ast.PatternLikeExpr{
- Expr: $1.(ast.ExprNode),
- Pattern: $4.(ast.ExprNode),
- Not: $2.(bool),
- Escape: escape[0],
- }
- }
- | PrimaryFactor NotOpt RegexpSym PrimaryExpression
- {
- $$ = &ast.PatternRegexpExpr{Expr: $1.(ast.ExprNode), Pattern: $4.(ast.ExprNode), Not: $2.(bool)}
- }
- | PrimaryFactor
-
- RegexpSym:
- "REGEXP"
- | "RLIKE"
-
- LikeEscapeOpt:
- %prec lowerThanEscape
- {
- $$ = "\\"
- }
- | "ESCAPE" stringLit
- {
- $$ = $2
- }
-
- NotOpt:
- {
- $$ = false
- }
- | "NOT"
- {
- $$ = true
- }
-
- Field:
- '*'
- {
- $$ = &ast.SelectField{WildCard: &ast.WildCardField{}}
- }
- | Identifier '.' '*'
- {
- wildCard := &ast.WildCardField{Table: model.NewCIStr($1.(string))}
- $$ = &ast.SelectField{WildCard: wildCard}
- }
- | Identifier '.' Identifier '.' '*'
- {
- wildCard := &ast.WildCardField{Schema: model.NewCIStr($1.(string)), Table: model.NewCIStr($3.(string))}
- $$ = &ast.SelectField{WildCard: wildCard}
- }
- | Expression FieldAsNameOpt
- {
- expr := $1.(ast.ExprNode)
- asName := $2.(string)
- $$ = &ast.SelectField{Expr: expr, AsName: model.NewCIStr(asName)}
- }
-
- FieldAsNameOpt:
- /* EMPTY */
- {
- $$ = ""
- }
- | FieldAsName
- {
- $$ = $1
- }
-
- FieldAsName:
- Identifier
- {
- $$ = $1
- }
- | "AS" Identifier
- {
- $$ = $2
- }
- | stringLit
- {
- $$ = $1
- }
- | "AS" stringLit
- {
- $$ = $2
- }
-
- FieldList:
- Field
- {
- field := $1.(*ast.SelectField)
- field.Offset = yylex.(*lexer).startOffset(yyS[yypt].offset)
- $$ = []*ast.SelectField{field}
- }
- | FieldList ',' Field
- {
-
- fl := $1.([]*ast.SelectField)
- last := fl[len(fl)-1]
- l := yylex.(*lexer)
- if last.Expr != nil && last.AsName.O == "" {
- lastEnd := l.endOffset(yyS[yypt-1].offset)
- last.SetText(l.src[last.Offset:lastEnd])
- }
- newField := $3.(*ast.SelectField)
- newField.Offset = l.startOffset(yyS[yypt].offset)
- $$ = append(fl, newField)
- }
-
- GroupByClause:
- "GROUP" "BY" ByList
- {
- $$ = &ast.GroupByClause{Items: $3.([]*ast.ByItem)}
- }
-
- HavingClause:
- {
- $$ = nil
- }
- | "HAVING" Expression
- {
- $$ = &ast.HavingClause{Expr: $2.(ast.ExprNode)}
- }
-
- IfExists:
- {
- $$ = false
- }
- | "IF" "EXISTS"
- {
- $$ = true
- }
-
- IfNotExists:
- {
- $$ = false
- }
- | "IF" "NOT" "EXISTS"
- {
- $$ = true
- }
-
-
- IgnoreOptional:
- {
- $$ = false
- }
- | "IGNORE"
- {
- $$ = true
- }
-
- IndexName:
- {
- $$ = ""
- }
- | Identifier
- {
- //"index name"
- $$ = $1.(string)
- }
-
- IndexOption:
- {
- $$ = nil
- }
- | "KEY_BLOCK_SIZE" EqOpt LengthNum
- {
- $$ = &ast.IndexOption{
- KeyBlockSize: $1.(uint64),
- }
- }
- | IndexType
- {
- $$ = &ast.IndexOption {
- Tp: $1.(model.IndexType),
- }
- }
- | "COMMENT" stringLit
- {
- $$ = &ast.IndexOption {
- Comment: $2.(string),
- }
- }
-
- IndexType:
- "USING" "BTREE"
- {
- $$ = model.IndexTypeBtree
- }
- | "USING" "HASH"
- {
- $$ = model.IndexTypeHash
- }
-
- IndexTypeOpt:
- {
- $$ = nil
- }
- | IndexType
- {
- $$ = $1
- }
-
- /**********************************Identifier********************************************/
- Identifier:
- identifier | UnReservedKeyword | NotKeywordToken
-
- UnReservedKeyword:
- "AUTO_INCREMENT" | "AFTER" | "AVG" | "BEGIN" | "BIT" | "BOOL" | "BOOLEAN" | "BTREE" | "CHARSET" | "COLUMNS" | "COMMIT" | "COMPACT" | "COMPRESSED"
- | "DATE" | "DATETIME" | "DEALLOCATE" | "DO" | "DYNAMIC" | "END" | "ENGINE" | "ENGINES" | "EXECUTE" | "FIRST" | "FIXED" | "FULL" | "HASH"
- | "LOCAL" | "NAMES" | "OFFSET" | "PASSWORD" %prec lowerThanEq | "PREPARE" | "QUICK" | "REDUNDANT" | "ROLLBACK" | "SESSION" | "SIGNED"
- | "START" | "STATUS" | "GLOBAL" | "TABLES"| "TEXT" | "TIME" | "TIMESTAMP" | "TRANSACTION" | "TRUNCATE" | "UNKNOWN"
- | "VALUE" | "WARNINGS" | "YEAR" | "MODE" | "WEEK" | "ANY" | "SOME" | "USER" | "IDENTIFIED" | "COLLATION"
- | "COMMENT" | "AVG_ROW_LENGTH" | "CONNECTION" | "CHECKSUM" | "COMPRESSION" | "KEY_BLOCK_SIZE" | "MAX_ROWS" | "MIN_ROWS"
- | "NATIONAL" | "ROW" | "ROW_FORMAT" | "QUARTER" | "ESCAPE" | "GRANTS" | "FIELDS" | "TRIGGERS" | "DELAY_KEY_WRITE" | "ISOLATION"
- | "REPEATABLE" | "COMMITTED" | "UNCOMMITTED" | "ONLY" | "SERIALIZABLE" | "LEVEL" | "VARIABLES"
-
- NotKeywordToken:
- "ABS" | "ADDDATE" | "ADMIN" | "COALESCE" | "CONCAT" | "CONCAT_WS" | "CONNECTION_ID" | "CUR_TIME"| "COUNT" | "DAY"
- | "DATE_ADD" | "DATE_SUB" | "DAYNAME" | "DAYOFMONTH" | "DAYOFWEEK" | "DAYOFYEAR" | "FOUND_ROWS" | "GROUP_CONCAT"| "HOUR"
- | "IFNULL" | "LENGTH" | "LOCATE" | "MAX" | "MICROSECOND" | "MIN" | "MINUTE" | "NULLIF" | "MONTH" | "NOW" | "POW"
- | "POWER" | "RAND" | "SECOND" | "SQL_CALC_FOUND_ROWS" | "SUBDATE" | "SUBSTRING" %prec lowerThanLeftParen
- | "SUBSTRING_INDEX" | "SUM" | "TRIM" | "VERSION" | "WEEKDAY" | "WEEKOFYEAR" | "YEARWEEK"
-
- /************************************************************************************
- *
- * Insert Statments
- *
- * TODO: support PARTITION
- **********************************************************************************/
- InsertIntoStmt:
- "INSERT" Priority IgnoreOptional IntoOpt TableName InsertValues OnDuplicateKeyUpdate
- {
- x := $6.(*ast.InsertStmt)
- x.Priority = $2.(int)
- // Wraps many layers here so that it can be processed the same way as select statement.
- ts := &ast.TableSource{Source: $5.(*ast.TableName)}
- x.Table = &ast.TableRefsClause{TableRefs: &ast.Join{Left: ts}}
- if $7 != nil {
- x.OnDuplicate = $7.([]*ast.Assignment)
- }
- $$ = x
- if yylex.(*lexer).root {
- break
- }
- }
-
- IntoOpt:
- {
- }
- | "INTO"
- {
- }
-
- InsertValues:
- '(' ColumnNameListOpt ')' ValueSym ExpressionListList
- {
- $$ = &ast.InsertStmt{
- Columns: $2.([]*ast.ColumnName),
- Lists: $5.([][]ast.ExprNode),
- }
- }
- | '(' ColumnNameListOpt ')' SelectStmt
- {
- $$ = &ast.InsertStmt{Columns: $2.([]*ast.ColumnName), Select: $4.(*ast.SelectStmt)}
- }
- | '(' ColumnNameListOpt ')' UnionStmt
- {
- $$ = &ast.InsertStmt{Columns: $2.([]*ast.ColumnName), Select: $4.(*ast.UnionStmt)}
- }
- | ValueSym ExpressionListList %prec insertValues
- {
- $$ = &ast.InsertStmt{Lists: $2.([][]ast.ExprNode)}
- }
- | SelectStmt
- {
- $$ = &ast.InsertStmt{Select: $1.(*ast.SelectStmt)}
- }
- | UnionStmt
- {
- $$ = &ast.InsertStmt{Select: $1.(*ast.UnionStmt)}
- }
- | "SET" ColumnSetValueList
- {
- $$ = &ast.InsertStmt{Setlist: $2.([]*ast.Assignment)}
- }
-
- ValueSym:
- "VALUE"
- | "VALUES"
-
- ExpressionListList:
- '(' ')'
- {
- $$ = [][]ast.ExprNode{[]ast.ExprNode{}}
- }
- | '(' ')' ',' ExpressionListList
- {
- $$ = append([][]ast.ExprNode{[]ast.ExprNode{}}, $4.([][]ast.ExprNode)...)
- }
- | '(' ExpressionList ')'
- {
- $$ = [][]ast.ExprNode{$2.([]ast.ExprNode)}
- }
- | '(' ExpressionList ')' ',' ExpressionListList
- {
- $$ = append([][]ast.ExprNode{$2.([]ast.ExprNode)}, $5.([][]ast.ExprNode)...)
- }
-
- ColumnSetValue:
- ColumnName eq Expression
- {
- $$ = &ast.Assignment{
- Column: $1.(*ast.ColumnName),
- Expr: $3.(ast.ExprNode),
- }
- }
-
- ColumnSetValueList:
- {
- $$ = []*ast.Assignment{}
- }
- | ColumnSetValue
- {
- $$ = []*ast.Assignment{$1.(*ast.Assignment)}
- }
- | ColumnSetValueList ',' ColumnSetValue
- {
- $$ = append($1.([]*ast.Assignment), $3.(*ast.Assignment))
- }
-
- /*
- * ON DUPLICATE KEY UPDATE col_name=expr [, col_name=expr] ...
- * See: https://dev.mysql.com/doc/refman/5.7/en/insert-on-duplicate.html
- */
- OnDuplicateKeyUpdate:
- {
- $$ = nil
- }
- | "ON" "DUPLICATE" "KEY" "UPDATE" AssignmentList
- {
- $$ = $5
- }
-
- /***********************************Insert Statements END************************************/
-
- /************************************************************************************
- * Replace Statements
- * See: https://dev.mysql.com/doc/refman/5.7/en/replace.html
- *
- * TODO: support PARTITION
- **********************************************************************************/
- ReplaceIntoStmt:
- "REPLACE" ReplacePriority IntoOpt TableName InsertValues
- {
- x := $5.(*ast.InsertStmt)
- x.IsReplace = true
- x.Priority = $2.(int)
- ts := &ast.TableSource{Source: $4.(*ast.TableName)}
- x.Table = &ast.TableRefsClause{TableRefs: &ast.Join{Left: ts}}
- $$ = x
- }
-
- ReplacePriority:
- {
- $$ = ast.NoPriority
- }
- | "LOW_PRIORITY"
- {
- $$ = ast.LowPriority
- }
- | "DELAYED"
- {
- $$ = ast.DelayedPriority
- }
-
- /***********************************Replace Statments END************************************/
-
- Literal:
- "false"
- {
- $$ = int64(0)
- }
- | "NULL"
- | "true"
- {
- $$ = int64(1)
- }
- | floatLit
- | intLit
- | stringLit
- {
- tp := types.NewFieldType(mysql.TypeString)
- l := yylex.(*lexer)
- tp.Charset, tp.Collate = l.GetCharsetInfo()
- expr := ast.NewValueExpr($1)
- expr.SetType(tp)
- $$ = expr
- }
- | "UNDERSCORE_CHARSET" stringLit
- {
- // See: https://dev.mysql.com/doc/refman/5.7/en/charset-literal.html
- tp := types.NewFieldType(mysql.TypeString)
- tp.Charset = $1.(string)
- co, err := charset.GetDefaultCollation(tp.Charset)
- if err != nil {
- l := yylex.(*lexer)
- l.errf("Get collation error for charset: %s", tp.Charset)
- return 1
- }
- tp.Collate = co
- expr := ast.NewValueExpr($2)
- expr.SetType(tp)
- $$ = expr
- }
- | hexLit
- | bitLit
-
- Operand:
- Literal
- {
- $$ = ast.NewValueExpr($1)
- }
- | ColumnName
- {
- $$ = &ast.ColumnNameExpr{Name: $1.(*ast.ColumnName)}
- }
- | '(' Expression ')'
- {
- l := yylex.(*lexer)
- startOffset := l.startOffset(yyS[yypt-1].offset)
- endOffset := l.endOffset(yyS[yypt].offset)
- expr := $2.(ast.ExprNode)
- expr.SetText(l.src[startOffset:endOffset])
- $$ = &ast.ParenthesesExpr{Expr: expr}
- }
- | "DEFAULT" %prec lowerThanLeftParen
- {
- $$ = &ast.DefaultExpr{}
- }
- | "DEFAULT" '(' ColumnName ')'
- {
- $$ = &ast.DefaultExpr{Name: $3.(*ast.ColumnName)}
- }
- | Variable
- {
- $$ = $1
- }
- | "PLACEHOLDER"
- {
- $$ = &ast.ParamMarkerExpr{
- Offset: yyS[yypt].offset,
- }
- }
- | "ROW" '(' Expression ',' ExpressionList ')'
- {
- values := append([]ast.ExprNode{$3.(ast.ExprNode)}, $5.([]ast.ExprNode)...)
- $$ = &ast.RowExpr{Values: values}
- }
- | '(' Expression ',' ExpressionList ')'
- {
- values := append([]ast.ExprNode{$2.(ast.ExprNode)}, $4.([]ast.ExprNode)...)
- $$ = &ast.RowExpr{Values: values}
- }
- | "EXISTS" SubSelect
- {
- $$ = &ast.ExistsSubqueryExpr{Sel: $2.(*ast.SubqueryExpr)}
- }
-
- OrderBy:
- "ORDER" "BY" ByList
- {
- $$ = &ast.OrderByClause{Items: $3.([]*ast.ByItem)}
- }
-
- ByList:
- ByItem
- {
- $$ = []*ast.ByItem{$1.(*ast.ByItem)}
- }
- | ByList ',' ByItem
- {
- $$ = append($1.([]*ast.ByItem), $3.(*ast.ByItem))
- }
-
- ByItem:
- Expression Order
- {
- expr := $1
- valueExpr, ok := expr.(*ast.ValueExpr)
- if ok {
- position, isPosition := valueExpr.GetValue().(int64)
- if isPosition {
- expr = &ast.PositionExpr{N: int(position)}
- }
- }
- $$ = &ast.ByItem{Expr: expr.(ast.ExprNode), Desc: $2.(bool)}
- }
-
- Order:
- /* EMPTY */
- {
- $$ = false // ASC by default
- }
- | "ASC"
- {
- $$ = false
- }
- | "DESC"
- {
- $$ = true
- }
-
- OrderByOptional:
- {
- $$ = nil
- }
- | OrderBy
- {
- $$ = $1
- }
-
- PrimaryExpression:
- Operand
- | Function
- | SubSelect
- | '!' PrimaryExpression %prec neg
- {
- $$ = &ast.UnaryOperationExpr{Op: opcode.Not, V: $2.(ast.ExprNode)}
- }
- | '~' PrimaryExpression %prec neg
- {
- $$ = &ast.UnaryOperationExpr{Op: opcode.BitNeg, V: $2.(ast.ExprNode)}
- }
- | '-' PrimaryExpression %prec neg
- {
- $$ = &ast.UnaryOperationExpr{Op: opcode.Minus, V: $2.(ast.ExprNode)}
- }
- | '+' PrimaryExpression %prec neg
- {
- $$ = &ast.UnaryOperationExpr{Op: opcode.Plus, V: $2.(ast.ExprNode)}
- }
- | "BINARY" PrimaryExpression %prec neg
- {
- // See: https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#operator_binary
- x := types.NewFieldType(mysql.TypeString)
- x.Charset = charset.CharsetBin
- x.Collate = charset.CharsetBin
- $$ = &ast.FuncCastExpr{
- Expr: $2.(ast.ExprNode),
- Tp: x,
- FunctionType: ast.CastBinaryOperator,
- }
- }
- | PrimaryExpression "COLLATE" StringName %prec neg
- {
- // TODO: Create a builtin function hold expr and collation. When do evaluation, convert expr result using the collation.
- $$ = $1
- }
-
- Function:
- FunctionCallKeyword
- | FunctionCallNonKeyword
- | FunctionCallConflict
- | FunctionCallAgg
-
- FunctionNameConflict:
- "DATABASE" | "SCHEMA" | "IF" | "LEFT" | "REPEAT" | "CURRENT_USER" | "CURRENT_DATE" | "VERSION"
-
- FunctionCallConflict:
- FunctionNameConflict '(' ExpressionListOpt ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: $3.([]ast.ExprNode)}
- }
- | "CURRENT_USER"
- {
- // See: https://dev.mysql.com/doc/refman/5.7/en/information-functions.html#function_current-user
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string))}
- }
- | "CURRENT_DATE"
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string))}
- }
-
- DistinctOpt:
- {
- $$ = false
- }
- | "ALL"
- {
- $$ = false
- }
- | "DISTINCT"
- {
- $$ = true
- }
- | "DISTINCT" "ALL"
- {
- $$ = true
- }
-
- FunctionCallKeyword:
- "CAST" '(' Expression "AS" CastType ')'
- {
- /* See: https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_cast */
- $$ = &ast.FuncCastExpr{
- Expr: $3.(ast.ExprNode),
- Tp: $5.(*types.FieldType),
- FunctionType: ast.CastFunction,
- }
- }
- | "CASE" ExpressionOpt WhenClauseList ElseOpt "END"
- {
- x := &ast.CaseExpr{WhenClauses: $3.([]*ast.WhenClause)}
- if $2 != nil {
- x.Value = $2.(ast.ExprNode)
- }
- if $4 != nil {
- x.ElseClause = $4.(ast.ExprNode)
- }
- $$ = x
- }
- | "CONVERT" '(' Expression "USING" StringName ')'
- {
- // See: https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert
- charset := ast.NewValueExpr($5)
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$3.(ast.ExprNode), charset},
- }
- }
- | "CONVERT" '(' Expression ',' CastType ')'
- {
- // See: https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert
- $$ = &ast.FuncCastExpr{
- Expr: $3.(ast.ExprNode),
- Tp: $5.(*types.FieldType),
- FunctionType: ast.CastConvertFunction,
- }
- }
- | "DATE" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "USER" '(' ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string))}
- }
- | "VALUES" '(' ColumnName ')' %prec lowerThanInsertValues
- {
- // TODO: support qualified identifier for column_name
- $$ = &ast.ValuesExpr{Column: &ast.ColumnNameExpr{Name: $3.(*ast.ColumnName)}}
- }
- | "WEEK" '(' ExpressionList ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: $3.([]ast.ExprNode)}
- }
- | "YEAR" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName:model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
-
- FunctionCallNonKeyword:
- "COALESCE" '(' ExpressionList ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: $3.([]ast.ExprNode)}
- }
- | "CURDATE" '(' ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string))}
- }
- | "CUR_TIME" '(' ExpressionOpt ')'
- {
- args := []ast.ExprNode{}
- if $3 != nil {
- args = append(args, $3.(ast.ExprNode))
- }
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: args}
- }
- | "CURRENT_TIME" FuncDatetimePrec
- {
- args := []ast.ExprNode{}
- if $2 != nil {
- args = append(args, $2.(ast.ExprNode))
- }
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: args}
- }
- | "CURRENT_TIMESTAMP" FuncDatetimePrec
- {
- args := []ast.ExprNode{}
- if $2 != nil {
- args = append(args, $2.(ast.ExprNode))
- }
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: args}
- }
- | "ABS" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "CONCAT" '(' ExpressionList ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: $3.([]ast.ExprNode)}
- }
- | "CONCAT_WS" '(' ExpressionList ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: $3.([]ast.ExprNode)}
- }
- | "DAY" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "DAYNAME" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "DAYOFWEEK" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "DAYOFMONTH" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "DAYOFYEAR" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | DateArithOpt '(' Expression ',' "INTERVAL" Expression TimeUnit ')'
- {
- op := ast.NewValueExpr($1)
- dateArithInterval := ast.NewValueExpr(
- ast.DateArithInterval{
- Unit: $7.(string),
- Interval: $6.(ast.ExprNode),
- },
- )
-
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr("DATE_ARITH"),
- Args: []ast.ExprNode{
- op,
- $3.(ast.ExprNode),
- dateArithInterval,
- },
- }
- }
- | DateArithMultiFormsOpt '(' Expression ',' DateArithInterval')'
- {
- op := ast.NewValueExpr($1)
- dateArithInterval := ast.NewValueExpr($5)
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr("DATE_ARITH"),
- Args: []ast.ExprNode{
- op,
- $3.(ast.ExprNode),
- dateArithInterval,
- },
- }
- }
- | "EXTRACT" '(' TimeUnit "FROM" Expression ')'
- {
- timeUnit := ast.NewValueExpr($3)
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{timeUnit, $5.(ast.ExprNode)},
- }
- }
- | "FOUND_ROWS" '(' ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string))}
- }
- | "HOUR" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "IFNULL" '(' ExpressionList ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: $3.([]ast.ExprNode)}
- }
- | "LENGTH" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "LOCATE" '(' Expression ',' Expression ')'
- {
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode)},
- }
- }
- | "LOCATE" '(' Expression ',' Expression ',' Expression ')'
- {
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode), $7.(ast.ExprNode)},
- }
- }
- | "LOWER" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "MICROSECOND" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "MINUTE" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "MONTH" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "NOW" '(' ExpressionOpt ')'
- {
- args := []ast.ExprNode{}
- if $3 != nil {
- args = append(args, $3.(ast.ExprNode))
- }
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: args}
- }
- | "NULLIF" '(' ExpressionList ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: $3.([]ast.ExprNode)}
- }
- | "POW" '(' Expression ',' Expression ')'
- {
- args := []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode)}
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: args}
- }
- | "POWER" '(' Expression ',' Expression ')'
- {
- args := []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode)}
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: args}
- }
- | "RAND" '(' ExpressionOpt ')'
- {
-
- args := []ast.ExprNode{}
- if $3 != nil {
- args = append(args, $3.(ast.ExprNode))
- }
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: args}
- }
- | "REPLACE" '(' Expression ',' Expression ',' Expression ')'
- {
- args := []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode), $7.(ast.ExprNode)}
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: args}
- }
- | "SECOND" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "STRCMP" '(' Expression ',' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode)}}
- }
- | "SUBSTRING" '(' Expression ',' Expression ')'
- {
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode)},
- }
- }
- | "SUBSTRING" '(' Expression "FROM" Expression ')'
- {
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode)},
- }
- }
- | "SUBSTRING" '(' Expression ',' Expression ',' Expression ')'
- {
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode), $7.(ast.ExprNode)},
- }
- }
- | "SUBSTRING" '(' Expression "FROM" Expression "FOR" Expression ')'
- {
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode), $7.(ast.ExprNode)},
- }
- }
- | "SUBSTRING_INDEX" '(' Expression ',' Expression ',' Expression ')'
- {
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$3.(ast.ExprNode), $5.(ast.ExprNode), $7.(ast.ExprNode)},
- }
- }
- | "SYSDATE" '(' ExpressionOpt ')'
- {
- args := []ast.ExprNode{}
- if $3 != nil {
- args = append(args, $3.(ast.ExprNode))
- }
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: args}
- }
- | "TRIM" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$3.(ast.ExprNode)},
- }
- }
- | "TRIM" '(' Expression "FROM" Expression ')'
- {
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$5.(ast.ExprNode), $3.(ast.ExprNode)},
- }
- }
- | "TRIM" '(' TrimDirection "FROM" Expression ')'
- {
- nilVal := ast.NewValueExpr(nil)
- direction := ast.NewValueExpr($3)
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$5.(ast.ExprNode), nilVal, direction},
- }
- }
- | "TRIM" '(' TrimDirection Expression "FROM" Expression ')'
- {
- direction := ast.NewValueExpr($3)
- $$ = &ast.FuncCallExpr{
- FnName: model.NewCIStr($1.(string)),
- Args: []ast.ExprNode{$6.(ast.ExprNode),$4.(ast.ExprNode), direction},
- }
- }
- | "UPPER" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "WEEKDAY" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "WEEKOFYEAR" '(' Expression ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
- }
- | "YEARWEEK" '(' ExpressionList ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string)), Args: $3.([]ast.ExprNode)}
- }
- | "CONNECTION_ID" '(' ')'
- {
- $$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1.(string))}
- }
-
- DateArithOpt:
- "DATE_ADD"
- {
- $$ = ast.DateAdd
- }
- | "DATE_SUB"
- {
- $$ = ast.DateSub
- }
-
- DateArithMultiFormsOpt:
- "ADDDATE"
- {
- $$ = ast.DateAdd
- }
- | "SUBDATE"
- {
- $$ = ast.DateSub
- }
-
- DateArithInterval:
- Expression
- {
- $$ = ast.DateArithInterval{
- Unit: "day",
- Interval: $1.(ast.ExprNode),
- }
- }
- | "INTERVAL" Expression TimeUnit
- {
- $$ = ast.DateArithInterval{Unit: $3.(string), Interval: $2.(ast.ExprNode)}
- }
-
- TrimDirection:
- "BOTH"
- {
- $$ = ast.TrimBoth
- }
- | "LEADING"
- {
- $$ = ast.TrimLeading
- }
- | "TRAILING"
- {
- $$ = ast.TrimTrailing
- }
-
- FunctionCallAgg:
- "AVG" '(' DistinctOpt ExpressionList ')'
- {
- $$ = &ast.AggregateFuncExpr{F: $1.(string), Args: $4.([]ast.ExprNode), Distinct: $3.(bool)}
- }
- | "COUNT" '(' DistinctOpt ExpressionList ')'
- {
- $$ = &ast.AggregateFuncExpr{F: $1.(string), Args: $4.([]ast.ExprNode), Distinct: $3.(bool)}
- }
- | "COUNT" '(' DistinctOpt '*' ')'
- {
- args := []ast.ExprNode{ast.NewValueExpr(ast.UnquoteString("*"))}
- $$ = &ast.AggregateFuncExpr{F: $1.(string), Args: args, Distinct: $3.(bool)}
- }
- | "GROUP_CONCAT" '(' DistinctOpt ExpressionList ')'
- {
- $$ = &ast.AggregateFuncExpr{F: $1.(string), Args: $4.([]ast.ExprNode), Distinct: $3.(bool)}
- }
- | "MAX" '(' DistinctOpt Expression ')'
- {
- $$ = &ast.AggregateFuncExpr{F: $1.(string), Args: []ast.ExprNode{$4.(ast.ExprNode)}, Distinct: $3.(bool)}
- }
- | "MIN" '(' DistinctOpt Expression ')'
- {
- $$ = &ast.AggregateFuncExpr{F: $1.(string), Args: []ast.ExprNode{$4.(ast.ExprNode)}, Distinct: $3.(bool)}
- }
- | "SUM" '(' DistinctOpt Expression ')'
- {
- $$ = &ast.AggregateFuncExpr{F: $1.(string), Args: []ast.ExprNode{$4.(ast.ExprNode)}, Distinct: $3.(bool)}
- }
-
- FuncDatetimePrec:
- {
- $$ = nil
- }
- | '(' ')'
- {
- $$ = nil
- }
- | '(' Expression ')'
- {
- $$ = $2
- }
-
- TimeUnit:
- "MICROSECOND" | "SECOND" | "MINUTE" | "HOUR" | "DAY" | "WEEK"
- | "MONTH" | "QUARTER" | "YEAR" | "SECOND_MICROSECOND" | "MINUTE_MICROSECOND"
- | "MINUTE_SECOND" | "HOUR_MICROSECOND" | "HOUR_SECOND" | "HOUR_MINUTE"
- | "DAY_MICROSECOND" | "DAY_SECOND" | "DAY_MINUTE" | "DAY_HOUR" | "YEAR_MONTH"
-
- ExpressionOpt:
- {
- $$ = nil
- }
- | Expression
- {
- $$ = $1
- }
-
- WhenClauseList:
- WhenClause
- {
- $$ = []*ast.WhenClause{$1.(*ast.WhenClause)}
- }
- | WhenClauseList WhenClause
- {
- $$ = append($1.([]*ast.WhenClause), $2.(*ast.WhenClause))
- }
-
- WhenClause:
- "WHEN" Expression "THEN" Expression
- {
- $$ = &ast.WhenClause{
- Expr: $2.(ast.ExprNode),
- Result: $4.(ast.ExprNode),
- }
- }
-
- ElseOpt:
- /* empty */
- {
- $$ = nil
- }
- | "ELSE" Expression
- {
- $$ = $2
- }
-
- CastType:
- "BINARY" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeString)
- x.Flen = $2.(int)
- x.Charset = charset.CharsetBin
- x.Collate = charset.CharsetBin
- $$ = x
- }
- | "CHAR" OptFieldLen OptBinary OptCharset
- {
- x := types.NewFieldType(mysql.TypeString)
- x.Flen = $2.(int)
- if $3.(bool) {
- x.Flag |= mysql.BinaryFlag
- }
- x.Charset = $4.(string)
- $$ = x
- }
- | "DATE"
- {
- x := types.NewFieldType(mysql.TypeDate)
- $$ = x
- }
- | "DATETIME" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeDatetime)
- x.Decimal = $2.(int)
- $$ = x
- }
- | "DECIMAL" FloatOpt
- {
- fopt := $2.(*ast.FloatOpt)
- x := types.NewFieldType(mysql.TypeNewDecimal)
- x.Flen = fopt.Flen
- x.Decimal = fopt.Decimal
- $$ = x
- }
- | "TIME" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeDuration)
- x.Decimal = $2.(int)
- $$ = x
- }
- | "SIGNED" OptInteger
- {
- x := types.NewFieldType(mysql.TypeLonglong)
- $$ = x
- }
- | "UNSIGNED" OptInteger
- {
- x := types.NewFieldType(mysql.TypeLonglong)
- x.Flag |= mysql.UnsignedFlag
- $$ = x
- }
-
-
- PrimaryFactor:
- PrimaryFactor '|' PrimaryFactor %prec '|'
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.Or, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor '&' PrimaryFactor %prec '&'
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.And, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor "<<" PrimaryFactor %prec lsh
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.LeftShift, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor ">>" PrimaryFactor %prec rsh
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.RightShift, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor '+' PrimaryFactor %prec '+'
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.Plus, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor '-' PrimaryFactor %prec '-'
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.Minus, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor '*' PrimaryFactor %prec '*'
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.Mul, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor '/' PrimaryFactor %prec '/'
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.Div, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor '%' PrimaryFactor %prec '%'
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.Mod, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor "DIV" PrimaryFactor %prec div
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.IntDiv, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor "MOD" PrimaryFactor %prec mod
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.Mod, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryFactor '^' PrimaryFactor
- {
- $$ = &ast.BinaryOperationExpr{Op: opcode.Xor, L: $1.(ast.ExprNode), R: $3.(ast.ExprNode)}
- }
- | PrimaryExpression
-
-
- Priority:
- {
- $$ = ast.NoPriority
- }
- | "LOW_PRIORITY"
- {
- $$ = ast.LowPriority
- }
- | "HIGH_PRIORITY"
- {
- $$ = ast.HighPriority
- }
- | "DELAYED"
- {
- $$ = ast.DelayedPriority
- }
-
- LowPriorityOptional:
- {
- $$ = false
- }
- | "LOW_PRIORITY"
- {
- $$ = true
- }
-
- TableName:
- Identifier
- {
- $$ = &ast.TableName{Name:model.NewCIStr($1.(string))}
- }
- | Identifier '.' Identifier
- {
- $$ = &ast.TableName{Schema:model.NewCIStr($1.(string)), Name:model.NewCIStr($3.(string))}
- }
-
- TableNameList:
- TableName
- {
- tbl := []*ast.TableName{$1.(*ast.TableName)}
- $$ = tbl
- }
- | TableNameList ',' TableName
- {
- $$ = append($1.([]*ast.TableName), $3.(*ast.TableName))
- }
-
- QuickOptional:
- %prec lowerThanQuick
- {
- $$ = false
- }
- | "QUICK"
- {
- $$ = true
- }
-
- /***************************Prepared Statement Start******************************
- * See: https://dev.mysql.com/doc/refman/5.7/en/prepare.html
- * Example:
- * PREPARE stmt_name FROM 'SELECT SQRT(POW(?,2) + POW(?,2)) AS hypotenuse';
- * OR
- * SET @s = 'SELECT SQRT(POW(?,2) + POW(?,2)) AS hypotenuse';
- * PREPARE stmt_name FROM @s;
- */
-
- PreparedStmt:
- "PREPARE" Identifier "FROM" PrepareSQL
- {
- var sqlText string
- var sqlVar *ast.VariableExpr
- switch $4.(type) {
- case string:
- sqlText = $4.(string)
- case *ast.VariableExpr:
- sqlVar = $4.(*ast.VariableExpr)
- }
- $$ = &ast.PrepareStmt{
- Name: $2.(string),
- SQLText: sqlText,
- SQLVar: sqlVar,
- }
- }
-
- PrepareSQL:
- stringLit
- | UserVariable
-
-
- /*
- * See: https://dev.mysql.com/doc/refman/5.7/en/execute.html
- * Example:
- * EXECUTE stmt1 USING @a, @b;
- * OR
- * EXECUTE stmt1;
- */
- ExecuteStmt:
- "EXECUTE" Identifier
- {
- $$ = &ast.ExecuteStmt{Name: $2.(string)}
- }
- | "EXECUTE" Identifier "USING" UserVariableList
- {
- $$ = &ast.ExecuteStmt{
- Name: $2.(string),
- UsingVars: $4.([]ast.ExprNode),
- }
- }
-
- UserVariableList:
- UserVariable
- {
- $$ = []ast.ExprNode{$1.(ast.ExprNode)}
- }
- | UserVariableList ',' UserVariable
- {
- $$ = append($1.([]ast.ExprNode), $3.(ast.ExprNode))
- }
-
- /*
- * See: https://dev.mysql.com/doc/refman/5.0/en/deallocate-prepare.html
- */
-
- DeallocateStmt:
- DeallocateSym "PREPARE" Identifier
- {
- $$ = &ast.DeallocateStmt{Name: $3.(string)}
- }
-
- DeallocateSym:
- "DEALLOCATE" | "DROP"
-
- /****************************Prepared Statement End*******************************/
-
-
- RollbackStmt:
- "ROLLBACK"
- {
- $$ = &ast.RollbackStmt{}
- }
-
- SelectStmt:
- "SELECT" SelectStmtOpts SelectStmtFieldList SelectStmtLimit SelectLockOpt
- {
- st := &ast.SelectStmt {
- Distinct: $2.(bool),
- Fields: $3.(*ast.FieldList),
- LockTp: $5.(ast.SelectLockType),
- }
- lastField := st.Fields.Fields[len(st.Fields.Fields)-1]
- if lastField.Expr != nil && lastField.AsName.O == "" {
- src := yylex.(*lexer).src
- var lastEnd int
- if $4 != nil {
- lastEnd = yyS[yypt-1].offset-1
- } else if $5 != ast.SelectLockNone {
- lastEnd = yyS[yypt].offset-1
- } else {
- lastEnd = len(src)
- if src[lastEnd-1] == ';' {
- lastEnd--
- }
- }
- lastField.SetText(src[lastField.Offset:lastEnd])
- }
- if $4 != nil {
- st.Limit = $4.(*ast.Limit)
- }
- $$ = st
- }
- | "SELECT" SelectStmtOpts SelectStmtFieldList FromDual WhereClauseOptional SelectStmtLimit SelectLockOpt
- {
- st := &ast.SelectStmt {
- Distinct: $2.(bool),
- Fields: $3.(*ast.FieldList),
- LockTp: $7.(ast.SelectLockType),
- }
- lastField := st.Fields.Fields[len(st.Fields.Fields)-1]
- if lastField.Expr != nil && lastField.AsName.O == "" {
- lastEnd := yyS[yypt-3].offset-1
- lastField.SetText(yylex.(*lexer).src[lastField.Offset:lastEnd])
- }
- if $5 != nil {
- st.Where = $5.(ast.ExprNode)
- }
- if $6 != nil {
- st.Limit = $6.(*ast.Limit)
- }
- $$ = st
- }
- | "SELECT" SelectStmtOpts SelectStmtFieldList "FROM"
- TableRefsClause WhereClauseOptional SelectStmtGroup HavingClause OrderByOptional
- SelectStmtLimit SelectLockOpt
- {
- st := &ast.SelectStmt{
- Distinct: $2.(bool),
- Fields: $3.(*ast.FieldList),
- From: $5.(*ast.TableRefsClause),
- LockTp: $11.(ast.SelectLockType),
- }
-
- lastField := st.Fields.Fields[len(st.Fields.Fields)-1]
- if lastField.Expr != nil && lastField.AsName.O == "" {
- lastEnd := yyS[yypt-7].offset-1
- lastField.SetText(yylex.(*lexer).src[lastField.Offset:lastEnd])
- }
-
- if $6 != nil {
- st.Where = $6.(ast.ExprNode)
- }
-
- if $7 != nil {
- st.GroupBy = $7.(*ast.GroupByClause)
- }
-
- if $8 != nil {
- st.Having = $8.(*ast.HavingClause)
- }
-
- if $9 != nil {
- st.OrderBy = $9.(*ast.OrderByClause)
- }
-
- if $10 != nil {
- st.Limit = $10.(*ast.Limit)
- }
-
- $$ = st
- }
-
- FromDual:
- "FROM" "DUAL"
-
-
- TableRefsClause:
- TableRefs
- {
- $$ = &ast.TableRefsClause{TableRefs: $1.(*ast.Join)}
- }
-
- TableRefs:
- EscapedTableRef
- {
- if j, ok := $1.(*ast.Join); ok {
- // if $1 is Join, use it directly
- $$ = j
- } else {
- $$ = &ast.Join{Left: $1.(ast.ResultSetNode), Right: nil}
- }
- }
- | TableRefs ',' EscapedTableRef
- {
- /* from a, b is default cross join */
- $$ = &ast.Join{Left: $1.(ast.ResultSetNode), Right: $3.(ast.ResultSetNode), Tp: ast.CrossJoin}
- }
-
- EscapedTableRef:
- TableRef %prec lowerThanSetKeyword
- {
- $$ = $1
- }
- | '{' Identifier TableRef '}'
- {
- /*
- * ODBC escape syntax for outer join is { OJ join_table }
- * Use an Identifier for OJ
- */
- $$ = $3
- }
-
- TableRef:
- TableFactor
- {
- $$ = $1
- }
- | JoinTable
- {
- $$ = $1
- }
-
- TableFactor:
- TableName TableAsNameOpt
- {
- $$ = &ast.TableSource{Source: $1.(*ast.TableName), AsName: $2.(model.CIStr)}
- }
- | '(' SelectStmt ')' TableAsName
- {
- st := $2.(*ast.SelectStmt)
- l := yylex.(*lexer)
- endOffset := l.endOffset(yyS[yypt-1].offset)
- l.SetLastSelectFieldText(st, endOffset)
- $$ = &ast.TableSource{Source: $2.(*ast.SelectStmt), AsName: $4.(model.CIStr)}
- }
- | '(' UnionStmt ')' TableAsName
- {
- $$ = &ast.TableSource{Source: $2.(*ast.UnionStmt), AsName: $4.(model.CIStr)}
- }
- | '(' TableRefs ')'
- {
- $$ = $2
- }
-
- TableAsNameOpt:
- {
- $$ = model.CIStr{}
- }
- | TableAsName
- {
- $$ = $1
- }
-
- TableAsName:
- Identifier
- {
- $$ = model.NewCIStr($1.(string))
- }
- | "AS" Identifier
- {
- $$ = model.NewCIStr($2.(string))
- }
-
- JoinTable:
- /* Use %prec to evaluate production TableRef before cross join */
- TableRef CrossOpt TableRef %prec tableRefPriority
- {
- $$ = &ast.Join{Left: $1.(ast.ResultSetNode), Right: $3.(ast.ResultSetNode), Tp: ast.CrossJoin}
- }
- | TableRef CrossOpt TableRef "ON" Expression
- {
- on := &ast.OnCondition{Expr: $5.(ast.ExprNode)}
- $$ = &ast.Join{Left: $1.(ast.ResultSetNode), Right: $3.(ast.ResultSetNode), Tp: ast.CrossJoin, On: on}
- }
- | TableRef JoinType OuterOpt "JOIN" TableRef "ON" Expression
- {
- on := &ast.OnCondition{Expr: $7.(ast.ExprNode)}
- $$ = &ast.Join{Left: $1.(ast.ResultSetNode), Right: $5.(ast.ResultSetNode), Tp: $2.(ast.JoinType), On: on}
- }
- /* Support Using */
-
- JoinType:
- "LEFT"
- {
- $$ = ast.LeftJoin
- }
- | "RIGHT"
- {
- $$ = ast.RightJoin
- }
-
- OuterOpt:
- {
- $$ = nil
- }
- | "OUTER"
-
-
- CrossOpt:
- "JOIN"
- | "CROSS" "JOIN"
- | "INNER" "JOIN"
-
-
- LimitClause:
- {
- $$ = nil
- }
- | "LIMIT" LengthNum
- {
- $$ = &ast.Limit{Count: $2.(uint64)}
- }
-
- SelectStmtLimit:
- {
- $$ = nil
- }
- | "LIMIT" LengthNum
- {
- $$ = &ast.Limit{Count: $2.(uint64)}
- }
- | "LIMIT" LengthNum ',' LengthNum
- {
- $$ = &ast.Limit{Offset: $2.(uint64), Count: $4.(uint64)}
- }
- | "LIMIT" LengthNum "OFFSET" LengthNum
- {
- $$ = &ast.Limit{Offset: $4.(uint64), Count: $2.(uint64)}
- }
-
- SelectStmtDistinct:
- /* EMPTY */
- {
- $$ = false
- }
- | "ALL"
- {
- $$ = false
- }
- | "DISTINCT"
- {
- $$ = true
- }
-
- SelectStmtOpts:
- SelectStmtDistinct SelectStmtCalcFoundRows
- {
- // TODO: return calc_found_rows opt and support more other options
- $$ = $1
- }
-
- SelectStmtCalcFoundRows:
- %prec lowerThanCalcFoundRows
- {
- $$ = false
- }
- | "SQL_CALC_FOUND_ROWS"
- {
- $$ = true
- }
-
- SelectStmtFieldList:
- FieldList
- {
- $$ = &ast.FieldList{Fields: $1.([]*ast.SelectField)}
- }
-
- SelectStmtGroup:
- /* EMPTY */
- {
- $$ = nil
- }
- | GroupByClause
-
- // See: https://dev.mysql.com/doc/refman/5.7/en/subqueries.html
- SubSelect:
- '(' SelectStmt ')'
- {
- s := $2.(*ast.SelectStmt)
- l := yylex.(*lexer)
- endOffset := l.endOffset(yyS[yypt].offset)
- l.SetLastSelectFieldText(s, endOffset)
- src := yylex.(*lexer).src
- // See the implemention of yyParse function
- s.SetText(src[yyS[yypt-1].offset-1:yyS[yypt].offset-1])
- $$ = &ast.SubqueryExpr{Query: s}
- }
- | '(' UnionStmt ')'
- {
- s := $2.(*ast.UnionStmt)
- src := yylex.(*lexer).src
- // See the implemention of yyParse function
- s.SetText(src[yyS[yypt-1].offset-1:yyS[yypt].offset-1])
- $$ = &ast.SubqueryExpr{Query: s}
- }
-
- // See: https://dev.mysql.com/doc/refman/5.7/en/innodb-locking-reads.html
- SelectLockOpt:
- /* empty */
- {
- $$ = ast.SelectLockNone
- }
- | "FOR" "UPDATE"
- {
- $$ = ast.SelectLockForUpdate
- }
- | "LOCK" "IN" "SHARE" "MODE"
- {
- $$ = ast.SelectLockInShareMode
- }
-
- // See: https://dev.mysql.com/doc/refman/5.7/en/union.html
- UnionStmt:
- UnionClauseList "UNION" UnionOpt SelectStmt
- {
- union := $1.(*ast.UnionStmt)
- union.Distinct = union.Distinct || $3.(bool)
- lastSelect := union.SelectList.Selects[len(union.SelectList.Selects)-1]
- l := yylex.(*lexer)
- endOffset := l.endOffset(yyS[yypt-2].offset)
- l.SetLastSelectFieldText(lastSelect, endOffset)
- union.SelectList.Selects = append(union.SelectList.Selects, $4.(*ast.SelectStmt))
- $$ = union
- }
- | UnionClauseList "UNION" UnionOpt '(' SelectStmt ')' OrderByOptional SelectStmtLimit
- {
- union := $1.(*ast.UnionStmt)
- union.Distinct = union.Distinct || $3.(bool)
- lastSelect := union.SelectList.Selects[len(union.SelectList.Selects)-1]
- l := yylex.(*lexer)
- endOffset := l.endOffset(yyS[yypt-6].offset)
- l.SetLastSelectFieldText(lastSelect, endOffset)
- st := $5.(*ast.SelectStmt)
- endOffset = l.endOffset(yyS[yypt-2].offset)
- l.SetLastSelectFieldText(st, endOffset)
- union.SelectList.Selects = append(union.SelectList.Selects, st)
- if $7 != nil {
- union.OrderBy = $7.(*ast.OrderByClause)
- }
- if $8 != nil {
- union.Limit = $8.(*ast.Limit)
- }
- $$ = union
- }
-
- UnionClauseList:
- UnionSelect
- {
- selectList := &ast.UnionSelectList{Selects: []*ast.SelectStmt{$1.(*ast.SelectStmt)}}
- $$ = &ast.UnionStmt{
- SelectList: selectList,
- }
- }
- | UnionClauseList "UNION" UnionOpt UnionSelect
- {
- union := $1.(*ast.UnionStmt)
- union.Distinct = union.Distinct || $3.(bool)
- lastSelect := union.SelectList.Selects[len(union.SelectList.Selects)-1]
- l := yylex.(*lexer)
- endOffset := l.endOffset(yyS[yypt-2].offset)
- l.SetLastSelectFieldText(lastSelect, endOffset)
- union.SelectList.Selects = append(union.SelectList.Selects, $4.(*ast.SelectStmt))
- $$ = union
- }
-
- UnionSelect:
- SelectStmt
- | '(' SelectStmt ')'
- {
- st := $2.(*ast.SelectStmt)
- l := yylex.(*lexer)
- endOffset := l.endOffset(yyS[yypt].offset)
- l.SetLastSelectFieldText(st, endOffset)
- $$ = st
- }
-
- UnionOpt:
- {
- $$ = true
- }
- | "ALL"
- {
- $$ = false
- }
- | "DISTINCT"
- {
- $$ = true
- }
-
-
- /********************Set Statement*******************************/
- SetStmt:
- "SET" VariableAssignmentList
- {
- $$ = &ast.SetStmt{Variables: $2.([]*ast.VariableAssignment)}
- }
- | "SET" "NAMES" StringName
- {
- $$ = &ast.SetCharsetStmt{Charset: $3.(string)}
- }
- | "SET" "NAMES" StringName "COLLATE" StringName
- {
- $$ = &ast.SetCharsetStmt{
- Charset: $3.(string),
- Collate: $5.(string),
- }
- }
- | "SET" CharsetKw StringName
- {
- $$ = &ast.SetCharsetStmt{Charset: $3.(string)}
- }
- | "SET" "PASSWORD" eq PasswordOpt
- {
- $$ = &ast.SetPwdStmt{Password: $4.(string)}
- }
- | "SET" "PASSWORD" "FOR" Username eq PasswordOpt
- {
- $$ = &ast.SetPwdStmt{User: $4.(string), Password: $6.(string)}
- }
- | "SET" "GLOBAL" "TRANSACTION" TransactionChars
- {
- // Parsed but ignored
- }
- | "SET" "SESSION" "TRANSACTION" TransactionChars
- {
- // Parsed but ignored
- }
-
- TransactionChars:
- TransactionChar
- | TransactionChars ',' TransactionChar
-
- TransactionChar:
- "ISOLATION" "LEVEL" IsolationLevel
- | "READ" "WRITE"
- | "READ" "ONLY"
-
- IsolationLevel:
- "REPEATABLE" "READ"
- | "READ" "COMMITTED"
- | "READ" "UNCOMMITTED"
- | "SERIALIZABLE"
-
- VariableAssignment:
- Identifier eq Expression
- {
- $$ = &ast.VariableAssignment{Name: $1.(string), Value: $3.(ast.ExprNode), IsSystem: true}
- }
- | "GLOBAL" Identifier eq Expression
- {
- $$ = &ast.VariableAssignment{Name: $2.(string), Value: $4.(ast.ExprNode), IsGlobal: true, IsSystem: true}
- }
- | "SESSION" Identifier eq Expression
- {
- $$ = &ast.VariableAssignment{Name: $2.(string), Value: $4.(ast.ExprNode), IsSystem: true}
- }
- | "LOCAL" Identifier eq Expression
- {
- $$ = &ast.VariableAssignment{Name: $2.(string), Value: $4.(ast.ExprNode), IsSystem: true}
- }
- | "SYS_VAR" eq Expression
- {
- v := strings.ToLower($1.(string))
- var isGlobal bool
- if strings.HasPrefix(v, "@@global.") {
- isGlobal = true
- v = strings.TrimPrefix(v, "@@global.")
- } else if strings.HasPrefix(v, "@@session.") {
- v = strings.TrimPrefix(v, "@@session.")
- } else if strings.HasPrefix(v, "@@local.") {
- v = strings.TrimPrefix(v, "@@local.")
- } else if strings.HasPrefix(v, "@@") {
- v = strings.TrimPrefix(v, "@@")
- }
- $$ = &ast.VariableAssignment{Name: v, Value: $3.(ast.ExprNode), IsGlobal: isGlobal, IsSystem: true}
- }
- | "USER_VAR" eq Expression
- {
- v := $1.(string)
- v = strings.TrimPrefix(v, "@")
- $$ = &ast.VariableAssignment{Name: v, Value: $3.(ast.ExprNode)}
- }
-
- VariableAssignmentList:
- {
- $$ = []*ast.VariableAssignment{}
- }
- | VariableAssignment
- {
- $$ = []*ast.VariableAssignment{$1.(*ast.VariableAssignment)}
- }
- | VariableAssignmentList ',' VariableAssignment
- {
- $$ = append($1.([]*ast.VariableAssignment), $3.(*ast.VariableAssignment))
- }
-
- Variable:
- SystemVariable | UserVariable
-
- SystemVariable:
- "SYS_VAR"
- {
- v := strings.ToLower($1.(string))
- var isGlobal bool
- if strings.HasPrefix(v, "@@global.") {
- isGlobal = true
- v = strings.TrimPrefix(v, "@@global.")
- } else if strings.HasPrefix(v, "@@session.") {
- v = strings.TrimPrefix(v, "@@session.")
- } else if strings.HasPrefix(v, "@@local.") {
- v = strings.TrimPrefix(v, "@@local.")
- } else if strings.HasPrefix(v, "@@") {
- v = strings.TrimPrefix(v, "@@")
- }
- $$ = &ast.VariableExpr{Name: v, IsGlobal: isGlobal, IsSystem: true}
- }
-
- UserVariable:
- "USER_VAR"
- {
- v := $1.(string)
- v = strings.TrimPrefix(v, "@")
- $$ = &ast.VariableExpr{Name: v, IsGlobal: false, IsSystem: false}
- }
-
- Username:
- stringLit "AT" stringLit
- {
- $$ = $1.(string) + "@" + $3.(string)
- }
-
- PasswordOpt:
- stringLit
- {
- $$ = $1.(string)
- }
- | "PASSWORD" '(' AuthString ')'
- {
- $$ = $3.(string)
- }
-
- AuthString:
- stringLit
- {
- $$ = $1.(string)
- }
-
- /****************************Admin Statement*******************************/
- AdminStmt:
- "ADMIN" "SHOW" "DDL"
- {
- $$ = &ast.AdminStmt{Tp: ast.AdminShowDDL}
- }
- | "ADMIN" "CHECK" "TABLE" TableNameList
- {
- $$ = &ast.AdminStmt{
- Tp: ast.AdminCheckTable,
- Tables: $4.([]*ast.TableName),
- }
- }
-
- /****************************Show Statement*******************************/
- ShowStmt:
- "SHOW" ShowTargetFilterable ShowLikeOrWhereOpt
- {
- stmt := $2.(*ast.ShowStmt)
- if $3 != nil {
- if x, ok := $3.(*ast.PatternLikeExpr); ok {
- stmt.Pattern = x
- } else {
- stmt.Where = $3.(ast.ExprNode)
- }
- }
- $$ = stmt
- }
- | "SHOW" "CREATE" "TABLE" TableName
- {
- $$ = &ast.ShowStmt{
- Tp: ast.ShowCreateTable,
- Table: $4.(*ast.TableName),
- }
- }
- | "SHOW" "GRANTS"
- {
- // See: https://dev.mysql.com/doc/refman/5.7/en/show-grants.html
- $$ = &ast.ShowStmt{Tp: ast.ShowGrants}
- }
- | "SHOW" "GRANTS" "FOR" Username
- {
- // See: https://dev.mysql.com/doc/refman/5.7/en/show-grants.html
- $$ = &ast.ShowStmt{
- Tp: ast.ShowGrants,
- User: $4.(string),
- }
- }
- | "SHOW" "INDEX" "FROM" TableName
- {
- $$ = &ast.ShowStmt{
- Tp: ast.ShowIndex,
- Table: $4.(*ast.TableName),
- }
- }
-
- ShowTargetFilterable:
- "ENGINES"
- {
- $$ = &ast.ShowStmt{Tp: ast.ShowEngines}
- }
- | "DATABASES"
- {
- $$ = &ast.ShowStmt{Tp: ast.ShowDatabases}
- }
- | "SCHEMAS"
- {
- $$ = &ast.ShowStmt{Tp: ast.ShowDatabases}
- }
- | "CHARACTER" "SET"
- {
- $$ = &ast.ShowStmt{Tp: ast.ShowCharset}
- }
- | OptFull "TABLES" ShowDatabaseNameOpt
- {
- $$ = &ast.ShowStmt{
- Tp: ast.ShowTables,
- DBName: $3.(string),
- Full: $1.(bool),
- }
- }
- | "TABLE" "STATUS" ShowDatabaseNameOpt
- {
- $$ = &ast.ShowStmt{
- Tp: ast.ShowTableStatus,
- DBName: $3.(string),
- }
- }
- | OptFull "COLUMNS" ShowTableAliasOpt ShowDatabaseNameOpt
- {
- $$ = &ast.ShowStmt{
- Tp: ast.ShowColumns,
- Table: $3.(*ast.TableName),
- DBName: $4.(string),
- Full: $1.(bool),
- }
- }
- | OptFull "FIELDS" ShowTableAliasOpt ShowDatabaseNameOpt
- {
- // SHOW FIELDS is a synonym for SHOW COLUMNS.
- $$ = &ast.ShowStmt{
- Tp: ast.ShowColumns,
- Table: $3.(*ast.TableName),
- DBName: $4.(string),
- Full: $1.(bool),
- }
- }
- | "WARNINGS"
- {
- $$ = &ast.ShowStmt{Tp: ast.ShowWarnings}
- }
- | GlobalScope "VARIABLES"
- {
- $$ = &ast.ShowStmt{
- Tp: ast.ShowVariables,
- GlobalScope: $1.(bool),
- }
- }
- | GlobalScope "STATUS"
- {
- $$ = &ast.ShowStmt{
- Tp: ast.ShowStatus,
- GlobalScope: $1.(bool),
- }
- }
- | "COLLATION"
- {
- $$ = &ast.ShowStmt{
- Tp: ast.ShowCollation,
- }
- }
- | "TRIGGERS" ShowDatabaseNameOpt
- {
- $$ = &ast.ShowStmt{
- Tp: ast.ShowTriggers,
- DBName: $2.(string),
- }
- }
- | "PROCEDURE" "STATUS"
- {
- $$ = &ast.ShowStmt {
- Tp: ast.ShowProcedureStatus,
- }
- }
-
- ShowLikeOrWhereOpt:
- {
- $$ = nil
- }
- | "LIKE" PrimaryExpression
- {
- $$ = &ast.PatternLikeExpr{Pattern: $2.(ast.ExprNode)}
- }
- | "WHERE" Expression
- {
- $$ = $2.(ast.ExprNode)
- }
-
- GlobalScope:
- {
- $$ = false
- }
- | "GLOBAL"
- {
- $$ = true
- }
- | "SESSION"
- {
- $$ = false
- }
-
- OptFull:
- {
- $$ = false
- }
- | "FULL"
- {
- $$ = true
- }
-
- ShowDatabaseNameOpt:
- {
- $$ = ""
- }
- | "FROM" DBName
- {
- $$ = $2.(string)
- }
- | "IN" DBName
- {
- $$ = $2.(string)
- }
-
- ShowTableAliasOpt:
- "FROM" TableName
- {
- $$ = $2.(*ast.TableName)
- }
- | "IN" TableName
- {
- $$ = $2.(*ast.TableName)
- }
-
- Statement:
- EmptyStmt
- | AdminStmt
- | AlterTableStmt
- | BeginTransactionStmt
- | CommitStmt
- | DeallocateStmt
- | DeleteFromStmt
- | ExecuteStmt
- | ExplainStmt
- | CreateDatabaseStmt
- | CreateIndexStmt
- | CreateTableStmt
- | CreateUserStmt
- | DoStmt
- | DropDatabaseStmt
- | DropIndexStmt
- | DropTableStmt
- | GrantStmt
- | InsertIntoStmt
- | PreparedStmt
- | RollbackStmt
- | ReplaceIntoStmt
- | SelectStmt
- | UnionStmt
- | SetStmt
- | ShowStmt
- | TruncateTableStmt
- | UpdateStmt
- | UseStmt
- | SubSelect
- {
- // `(select 1)`; is a valid select statement
- // TODO: This is used to fix issue #320. There may be a better solution.
- $$ = $1.(*ast.SubqueryExpr).Query
- }
- | UnlockTablesStmt
- | LockTablesStmt
-
- ExplainableStmt:
- SelectStmt
- | DeleteFromStmt
- | UpdateStmt
- | InsertIntoStmt
- | ReplaceIntoStmt
-
- StatementList:
- Statement
- {
- if $1 != nil {
- s := $1.(ast.StmtNode)
- s.SetText(yylex.(*lexer).stmtText())
- yylex.(*lexer).list = append(yylex.(*lexer).list, s)
- }
- }
- | StatementList ';' Statement
- {
- if $3 != nil {
- s := $3.(ast.StmtNode)
- s.SetText(yylex.(*lexer).stmtText())
- yylex.(*lexer).list = append(yylex.(*lexer).list, s)
- }
- }
-
- Constraint:
- ConstraintKeywordOpt ConstraintElem
- {
- cst := $2.(*ast.Constraint)
- if $1 != nil {
- cst.Name = $1.(string)
- }
- $$ = cst
- }
-
- TableElement:
- ColumnDef
- {
- $$ = $1.(*ast.ColumnDef)
- }
- | Constraint
- {
- $$ = $1.(*ast.Constraint)
- }
- | "CHECK" '(' Expression ')'
- {
- /* Nothing to do now */
- $$ = nil
- }
-
- TableElementList:
- TableElement
- {
- if $1 != nil {
- $$ = []interface{}{$1.(interface{})}
- } else {
- $$ = []interface{}{}
- }
- }
- | TableElementList ',' TableElement
- {
- if $3 != nil {
- $$ = append($1.([]interface{}), $3)
- } else {
- $$ = $1
- }
- }
-
- TableOption:
- "ENGINE" Identifier
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionEngine, StrValue: $2.(string)}
- }
- | "ENGINE" eq Identifier
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionEngine, StrValue: $3.(string)}
- }
- | DefaultKwdOpt CharsetKw EqOpt StringName
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionCharset, StrValue: $4.(string)}
- }
- | DefaultKwdOpt "COLLATE" EqOpt StringName
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionCollate, StrValue: $4.(string)}
- }
- | "AUTO_INCREMENT" eq LengthNum
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionAutoIncrement, UintValue: $3.(uint64)}
- }
- | "COMMENT" EqOpt stringLit
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionComment, StrValue: $3.(string)}
- }
- | "AVG_ROW_LENGTH" EqOpt LengthNum
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionAvgRowLength, UintValue: $3.(uint64)}
- }
- | "CONNECTION" EqOpt stringLit
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionConnection, StrValue: $3.(string)}
- }
- | "CHECKSUM" EqOpt LengthNum
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionCheckSum, UintValue: $3.(uint64)}
- }
- | "PASSWORD" EqOpt stringLit
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionPassword, StrValue: $3.(string)}
- }
- | "COMPRESSION" EqOpt Identifier
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionCompression, StrValue: $3.(string)}
- }
- | "KEY_BLOCK_SIZE" EqOpt LengthNum
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionKeyBlockSize, UintValue: $3.(uint64)}
- }
- | "MAX_ROWS" EqOpt LengthNum
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionMaxRows, UintValue: $3.(uint64)}
- }
- | "MIN_ROWS" EqOpt LengthNum
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionMinRows, UintValue: $3.(uint64)}
- }
- | "DELAY_KEY_WRITE" EqOpt LengthNum
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionDelayKeyWrite, UintValue: $3.(uint64)}
- }
- | RowFormat
- {
- $$ = &ast.TableOption{Tp: ast.TableOptionRowFormat, UintValue: $1.(uint64)}
- }
-
-
- TableOptionListOpt:
- {
- $$ = []*ast.TableOption{}
- }
- | TableOptionList %prec lowerThanComma
-
- TableOptionList:
- TableOption
- {
- $$ = []*ast.TableOption{$1.(*ast.TableOption)}
- }
- | TableOptionList TableOption
- {
- $$ = append($1.([]*ast.TableOption), $2.(*ast.TableOption))
- }
- | TableOptionList ',' TableOption
- {
- $$ = append($1.([]*ast.TableOption), $3.(*ast.TableOption))
- }
-
-
- TruncateTableStmt:
- "TRUNCATE" "TABLE" TableName
- {
- $$ = &ast.TruncateTableStmt{Table: $3.(*ast.TableName)}
- }
-
- RowFormat:
- "ROW_FORMAT" EqOpt "DEFAULT"
- {
- $$ = ast.RowFormatDefault
- }
- | "ROW_FORMAT" EqOpt "DYNAMIC"
- {
- $$ = ast.RowFormatDynamic
- }
- | "ROW_FORMAT" EqOpt "FIXED"
- {
- $$ = ast.RowFormatFixed
- }
- | "ROW_FORMAT" EqOpt "COMPRESSED"
- {
- $$ = ast.RowFormatCompressed
- }
- | "ROW_FORMAT" EqOpt "REDUNDANT"
- {
- $$ = ast.RowFormatRedundant
- }
- | "ROW_FORMAT" EqOpt "COMPACT"
- {
- $$ = ast.RowFormatCompact
- }
-
- /*************************************Type Begin***************************************/
- Type:
- NumericType
- {
- $$ = $1
- }
- | StringType
- {
- $$ = $1
- }
- | DateAndTimeType
- {
- $$ = $1
- }
- | "float32"
- {
- x := types.NewFieldType($1.(byte))
- $$ = x
- }
- | "float64"
- {
- x := types.NewFieldType($1.(byte))
- $$ = x
- }
- | "int64"
- {
- x := types.NewFieldType($1.(byte))
- $$ = x
- }
- | "string"
- {
- x := types.NewFieldType($1.(byte))
- $$ = x
- }
- | "uint"
- {
- x := types.NewFieldType($1.(byte))
- $$ = x
- }
- | "uint64"
- {
- x := types.NewFieldType($1.(byte))
- $$ = x
- }
-
- NumericType:
- IntegerType OptFieldLen FieldOpts
- {
- // TODO: check flen 0
- x := types.NewFieldType($1.(byte))
- x.Flen = $2.(int)
- for _, o := range $3.([]*ast.TypeOpt) {
- if o.IsUnsigned {
- x.Flag |= mysql.UnsignedFlag
- }
- if o.IsZerofill {
- x.Flag |= mysql.ZerofillFlag
- }
- }
- $$ = x
- }
- | FixedPointType FloatOpt FieldOpts
- {
- fopt := $2.(*ast.FloatOpt)
- x := types.NewFieldType($1.(byte))
- x.Flen = fopt.Flen
- x.Decimal = fopt.Decimal
- for _, o := range $3.([]*ast.TypeOpt) {
- if o.IsUnsigned {
- x.Flag |= mysql.UnsignedFlag
- }
- if o.IsZerofill {
- x.Flag |= mysql.ZerofillFlag
- }
- }
- $$ = x
- }
- | FloatingPointType FloatOpt FieldOpts
- {
- fopt := $2.(*ast.FloatOpt)
- x := types.NewFieldType($1.(byte))
- x.Flen = fopt.Flen
- if x.Tp == mysql.TypeFloat {
- // Fix issue #312
- if x.Flen > 53 {
- yylex.(*lexer).errf("Float len(%d) should not be greater than 53", x.Flen)
- return 1
- }
- if x.Flen > 24 {
- x.Tp = mysql.TypeDouble
- }
- }
- x.Decimal =fopt.Decimal
- for _, o := range $3.([]*ast.TypeOpt) {
- if o.IsUnsigned {
- x.Flag |= mysql.UnsignedFlag
- }
- if o.IsZerofill {
- x.Flag |= mysql.ZerofillFlag
- }
- }
- $$ = x
- }
- | BitValueType OptFieldLen
- {
- x := types.NewFieldType($1.(byte))
- x.Flen = $2.(int)
- if x.Flen == -1 || x.Flen == 0 {
- x.Flen = 1
- } else if x.Flen > 64 {
- yylex.(*lexer).errf("invalid field length %d for bit type, must in [1, 64]", x.Flen)
- }
- $$ = x
- }
-
- IntegerType:
- "TINYINT"
- {
- $$ = mysql.TypeTiny
- }
- | "SMALLINT"
- {
- $$ = mysql.TypeShort
- }
- | "MEDIUMINT"
- {
- $$ = mysql.TypeInt24
- }
- | "INT"
- {
- $$ = mysql.TypeLong
- }
- | "INTEGER"
- {
- $$ = mysql.TypeLong
- }
- | "BIGINT"
- {
- $$ = mysql.TypeLonglong
- }
- | "BOOL"
- {
- $$ = mysql.TypeTiny
- }
- | "BOOLEAN"
- {
- $$ = mysql.TypeTiny
- }
-
- OptInteger:
- {} | "INTEGER"
-
- FixedPointType:
- "DECIMAL"
- {
- $$ = mysql.TypeNewDecimal
- }
- | "NUMERIC"
- {
- $$ = mysql.TypeNewDecimal
- }
-
- FloatingPointType:
- "float"
- {
- $$ = mysql.TypeFloat
- }
- | "REAL"
- {
- $$ = mysql.TypeDouble
- }
- | "DOUBLE"
- {
- $$ = mysql.TypeDouble
- }
- | "DOUBLE" "PRECISION"
- {
- $$ = mysql.TypeDouble
- }
-
- BitValueType:
- "BIT"
- {
- $$ = mysql.TypeBit
- }
-
- StringType:
- NationalOpt "CHAR" FieldLen OptBinary OptCharset OptCollate
- {
- x := types.NewFieldType(mysql.TypeString)
- x.Flen = $3.(int)
- if $4.(bool) {
- x.Flag |= mysql.BinaryFlag
- }
- $$ = x
- }
- | NationalOpt "CHAR" OptBinary OptCharset OptCollate
- {
- x := types.NewFieldType(mysql.TypeString)
- if $3.(bool) {
- x.Flag |= mysql.BinaryFlag
- }
- $$ = x
- }
- | NationalOpt "VARCHAR" FieldLen OptBinary OptCharset OptCollate
- {
- x := types.NewFieldType(mysql.TypeVarchar)
- x.Flen = $3.(int)
- if $4.(bool) {
- x.Flag |= mysql.BinaryFlag
- }
- x.Charset = $5.(string)
- x.Collate = $6.(string)
- $$ = x
- }
- | "BINARY" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeString)
- x.Flen = $2.(int)
- x.Charset = charset.CharsetBin
- x.Collate = charset.CharsetBin
- $$ = x
- }
- | "VARBINARY" FieldLen
- {
- x := types.NewFieldType(mysql.TypeVarchar)
- x.Flen = $2.(int)
- x.Charset = charset.CharsetBin
- x.Collate = charset.CharsetBin
- $$ = x
- }
- | BlobType
- {
- $$ = $1.(*types.FieldType)
- }
- | TextType OptBinary OptCharset OptCollate
- {
- x := $1.(*types.FieldType)
- if $2.(bool) {
- x.Flag |= mysql.BinaryFlag
- }
- x.Charset = $3.(string)
- x.Collate = $4.(string)
- $$ = x
- }
- | "ENUM" '(' StringList ')' OptCharset OptCollate
- {
- x := types.NewFieldType(mysql.TypeEnum)
- x.Elems = $3.([]string)
- x.Charset = $5.(string)
- x.Collate = $6.(string)
- $$ = x
- }
- | "SET" '(' StringList ')' OptCharset OptCollate
- {
- x := types.NewFieldType(mysql.TypeSet)
- x.Elems = $3.([]string)
- x.Charset = $5.(string)
- x.Collate = $6.(string)
- $$ = x
- }
-
- NationalOpt:
- {
-
- }
- | "NATIONAL"
- {
-
- }
-
- BlobType:
- "TINYBLOB"
- {
- x := types.NewFieldType(mysql.TypeTinyBlob)
- x.Charset = charset.CharsetBin
- x.Collate = charset.CharsetBin
- $$ = x
- }
- | "BLOB" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeBlob)
- x.Flen = $2.(int)
- x.Charset = charset.CharsetBin
- x.Collate = charset.CharsetBin
- $$ = x
- }
- | "MEDIUMBLOB"
- {
- x := types.NewFieldType(mysql.TypeMediumBlob)
- x.Charset = charset.CharsetBin
- x.Collate = charset.CharsetBin
- $$ = x
- }
- | "LONGBLOB"
- {
- x := types.NewFieldType(mysql.TypeLongBlob)
- x.Charset = charset.CharsetBin
- x.Collate = charset.CharsetBin
- $$ = x
- }
-
- TextType:
- "TINYTEXT"
- {
- x := types.NewFieldType(mysql.TypeTinyBlob)
- $$ = x
-
- }
- | "TEXT" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeBlob)
- x.Flen = $2.(int)
- $$ = x
- }
- | "MEDIUMTEXT"
- {
- x := types.NewFieldType(mysql.TypeMediumBlob)
- $$ = x
- }
- | "LONGTEXT"
- {
- x := types.NewFieldType(mysql.TypeLongBlob)
- $$ = x
- }
-
-
- DateAndTimeType:
- "DATE"
- {
- x := types.NewFieldType(mysql.TypeDate)
- $$ = x
- }
- | "DATETIME" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeDatetime)
- x.Decimal = $2.(int)
- $$ = x
- }
- | "TIMESTAMP" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeTimestamp)
- x.Decimal = $2.(int)
- $$ = x
- }
- | "TIME" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeDuration)
- x.Decimal = $2.(int)
- $$ = x
- }
- | "YEAR" OptFieldLen
- {
- x := types.NewFieldType(mysql.TypeYear)
- x.Flen = $2.(int)
- $$ = x
- }
-
- FieldLen:
- '(' LengthNum ')'
- {
- $$ = int($2.(uint64))
- }
-
- OptFieldLen:
- {
- /* -1 means unspecified field length*/
- $$ = types.UnspecifiedLength
- }
- | FieldLen
- {
- $$ = $1.(int)
- }
-
- FieldOpt:
- "UNSIGNED"
- {
- $$ = &ast.TypeOpt{IsUnsigned: true}
- }
- | "ZEROFILL"
- {
- $$ = &ast.TypeOpt{IsZerofill: true, IsUnsigned: true}
- }
-
- FieldOpts:
- {
- $$ = []*ast.TypeOpt{}
- }
- | FieldOpts FieldOpt
- {
- $$ = append($1.([]*ast.TypeOpt), $2.(*ast.TypeOpt))
- }
-
- FloatOpt:
- {
- $$ = &ast.FloatOpt{Flen: types.UnspecifiedLength, Decimal: types.UnspecifiedLength}
- }
- | FieldLen
- {
- $$ = &ast.FloatOpt{Flen: $1.(int), Decimal: types.UnspecifiedLength}
- }
- | Precision
- {
- $$ = $1.(*ast.FloatOpt)
- }
-
- Precision:
- '(' LengthNum ',' LengthNum ')'
- {
- $$ = &ast.FloatOpt{Flen: int($2.(uint64)), Decimal: int($4.(uint64))}
- }
-
- OptBinary:
- {
- $$ = false
- }
- | "BINARY"
- {
- $$ = true
- }
-
- OptCharset:
- {
- $$ = ""
- }
- | CharsetKw StringName
- {
- $$ = $2.(string)
- }
-
- CharsetKw:
- "CHARACTER" "SET"
- | "CHARSET"
-
- OptCollate:
- {
- $$ = ""
- }
- | "COLLATE" StringName
- {
- $$ = $2.(string)
- }
-
- StringList:
- stringLit
- {
- $$ = []string{$1.(string)}
- }
- | StringList ',' stringLit
- {
- $$ = append($1.([]string), $3.(string))
- }
-
- StringName:
- stringLit
- {
- $$ = $1.(string)
- }
- | Identifier
- {
- $$ = $1.(string)
- }
-
- /***********************************************************************************
- * Update Statement
- * See: https://dev.mysql.com/doc/refman/5.7/en/update.html
- ***********************************************************************************/
- UpdateStmt:
- "UPDATE" LowPriorityOptional IgnoreOptional TableRef "SET" AssignmentList WhereClauseOptional OrderByOptional LimitClause
- {
- var refs *ast.Join
- if x, ok := $4.(*ast.Join); ok {
- refs = x
- } else {
- refs = &ast.Join{Left: $4.(ast.ResultSetNode)}
- }
- st := &ast.UpdateStmt{
- LowPriority: $2.(bool),
- TableRefs: &ast.TableRefsClause{TableRefs: refs},
- List: $6.([]*ast.Assignment),
- }
- if $7 != nil {
- st.Where = $7.(ast.ExprNode)
- }
- if $8 != nil {
- st.Order = $8.(*ast.OrderByClause)
- }
- if $9 != nil {
- st.Limit = $9.(*ast.Limit)
- }
- $$ = st
- if yylex.(*lexer).root {
- break
- }
- }
- | "UPDATE" LowPriorityOptional IgnoreOptional TableRefs "SET" AssignmentList WhereClauseOptional
- {
- st := &ast.UpdateStmt{
- LowPriority: $2.(bool),
- TableRefs: &ast.TableRefsClause{TableRefs: $4.(*ast.Join)},
- List: $6.([]*ast.Assignment),
- }
- if $7 != nil {
- st.Where = $7.(ast.ExprNode)
- }
- $$ = st
- if yylex.(*lexer).root {
- break
- }
- }
-
- UseStmt:
- "USE" DBName
- {
- $$ = &ast.UseStmt{DBName: $2.(string)}
- if yylex.(*lexer).root {
- break
- }
- }
-
- WhereClause:
- "WHERE" Expression
- {
- $$ = $2.(ast.ExprNode)
- }
-
- WhereClauseOptional:
- {
- $$ = nil
- }
- | WhereClause
- {
- $$ = $1
- }
-
- CommaOpt:
- {
- }
- | ','
- {
- }
-
- /************************************************************************************
- * Account Management Statements
- * https://dev.mysql.com/doc/refman/5.7/en/account-management-sql.html
- ************************************************************************************/
- CreateUserStmt:
- "CREATE" "USER" IfNotExists UserSpecList
- {
- // See: https://dev.mysql.com/doc/refman/5.7/en/create-user.html
- $$ = &ast.CreateUserStmt{
- IfNotExists: $3.(bool),
- Specs: $4.([]*ast.UserSpec),
- }
- }
-
- UserSpec:
- Username AuthOption
- {
- userSpec := &ast.UserSpec{
- User: $1.(string),
- }
- if $2 != nil {
- userSpec.AuthOpt = $2.(*ast.AuthOption)
- }
- $$ = userSpec
- }
-
- UserSpecList:
- UserSpec
- {
- $$ = []*ast.UserSpec{$1.(*ast.UserSpec)}
- }
- | UserSpecList ',' UserSpec
- {
- $$ = append($1.([]*ast.UserSpec), $3.(*ast.UserSpec))
- }
-
- AuthOption:
- {
- $$ = nil
- }
- | "IDENTIFIED" "BY" AuthString
- {
- $$ = &ast.AuthOption {
- AuthString: $3.(string),
- ByAuthString: true,
- }
- }
- | "IDENTIFIED" "BY" "PASSWORD" HashString
- {
- $$ = &ast.AuthOption{
- HashString: $4.(string),
- }
- }
-
- HashString:
- stringLit
-
- /*************************************************************************************
- * Grant statement
- * See: https://dev.mysql.com/doc/refman/5.7/en/grant.html
- *************************************************************************************/
- GrantStmt:
- "GRANT" PrivElemList "ON" ObjectType PrivLevel "TO" UserSpecList
- {
- $$ = &ast.GrantStmt{
- Privs: $2.([]*ast.PrivElem),
- ObjectType: $4.(ast.ObjectTypeType),
- Level: $5.(*ast.GrantLevel),
- Users: $7.([]*ast.UserSpec),
- }
- }
-
- PrivElem:
- PrivType
- {
- $$ = &ast.PrivElem{
- Priv: $1.(mysql.PrivilegeType),
- }
- }
- | PrivType '(' ColumnNameList ')'
- {
- $$ = &ast.PrivElem{
- Priv: $1.(mysql.PrivilegeType),
- Cols: $3.([]*ast.ColumnName),
- }
- }
-
- PrivElemList:
- PrivElem
- {
- $$ = []*ast.PrivElem{$1.(*ast.PrivElem)}
- }
- | PrivElemList ',' PrivElem
- {
- $$ = append($1.([]*ast.PrivElem), $3.(*ast.PrivElem))
- }
-
- PrivType:
- "ALL"
- {
- $$ = mysql.AllPriv
- }
- | "ALTER"
- {
- $$ = mysql.AlterPriv
- }
- | "CREATE"
- {
- $$ = mysql.CreatePriv
- }
- | "CREATE" "USER"
- {
- $$ = mysql.CreateUserPriv
- }
- | "DELETE"
- {
- $$ = mysql.DeletePriv
- }
- | "DROP"
- {
- $$ = mysql.DropPriv
- }
- | "EXECUTE"
- {
- $$ = mysql.ExecutePriv
- }
- | "INDEX"
- {
- $$ = mysql.IndexPriv
- }
- | "INSERT"
- {
- $$ = mysql.InsertPriv
- }
- | "SELECT"
- {
- $$ = mysql.SelectPriv
- }
- | "SHOW" "DATABASES"
- {
- $$ = mysql.ShowDBPriv
- }
- | "UPDATE"
- {
- $$ = mysql.UpdatePriv
- }
- | "GRANT" "OPTION"
- {
- $$ = mysql.GrantPriv
- }
-
- ObjectType:
- {
- $$ = ast.ObjectTypeNone
- }
- | "TABLE"
- {
- $$ = ast.ObjectTypeTable
- }
-
- PrivLevel:
- '*'
- {
- $$ = &ast.GrantLevel {
- Level: ast.GrantLevelDB,
- }
- }
- | '*' '.' '*'
- {
- $$ = &ast.GrantLevel {
- Level: ast.GrantLevelGlobal,
- }
- }
- | Identifier '.' '*'
- {
- $$ = &ast.GrantLevel {
- Level: ast.GrantLevelDB,
- DBName: $1.(string),
- }
- }
- | Identifier '.' Identifier
- {
- $$ = &ast.GrantLevel {
- Level: ast.GrantLevelTable,
- DBName: $1.(string),
- TableName: $3.(string),
- }
- }
- | Identifier
- {
- $$ = &ast.GrantLevel {
- Level: ast.GrantLevelTable,
- TableName: $1.(string),
- }
- }
-
- /*********************************************************************
- * Lock/Unlock Tables
- * See: http://dev.mysql.com/doc/refman/5.7/en/lock-tables.html
- * All the statement leaves empty. This is used to prevent mysqldump error.
- *********************************************************************/
-
- UnlockTablesStmt:
- "UNLOCK" "TABLES"
-
- LockTablesStmt:
- "LOCK" "TABLES" TableLockList
-
- TableLock:
- TableName LockType
-
- LockType:
- "READ"
- | "READ" "LOCAL"
- | "WRITE"
-
- TableLockList:
- TableLock
- | TableLockList ',' TableLock
-
- %%
|