diff --git a/ast/ast.go b/ast/ast.go new file mode 100644 index 0000000000..5caa14a8ae --- /dev/null +++ b/ast/ast.go @@ -0,0 +1,81 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +// Package ast is the abstract syntax tree parsed from a SQL statement by parser. +// It can be analysed and transformed by optimizer. +package ast + +import ( + "github.com/pingcap/tidb/util/types" +) + +// Node is the basic element of the AST. +// Interfaces embed Node should have 'Node' name suffix. +type Node interface { + // Accept accepts Visitor to visit itself. + // The returned node should replace original node. + // ok returns false to stop visiting. + Accept(v Visitor) (node Node, ok bool) + // Text returns the original text of the element. + Text() string + // SetText sets original text to the Node. + SetText(text string) +} + +// ExprNode is a node that can be evaluated. +// Name of implementations should have 'Expr' suffix. +type ExprNode interface { + // Node is embeded in ExprNode. + Node + // IsStatic means it can be evaluated independently. + IsStatic() bool + // SetType sets evaluation type to the expression. + SetType(tp *types.FieldType) + // GetType gets the evaluation type of the expression. + GetType() *types.FieldType +} + +// FuncNode represents function call expression node. +type FuncNode interface { + ExprNode + functionExpression() +} + +// StmtNode represents statement node. +// Name of implementations should have 'Stmt' suffix. +type StmtNode interface { + Node + statement() +} + +// DDLNode represents DDL statement node. +type DDLNode interface { + StmtNode + ddlStatement() +} + +// DMLNode represents DML statement node. +type DMLNode interface { + StmtNode + dmlStatement() +} + +// Visitor visits a Node. +type Visitor interface { + // VisitEnter is called before children nodes is visited. + // ok returns false to stop visiting. + Enter(n Node) (ok bool) + // VisitLeave is called after children nodes has been visited. + // ok returns false to stop visiting. + Leave(n Node) (node Node, ok bool) +} diff --git a/ast/base.go b/ast/base.go new file mode 100644 index 0000000000..5767cd9ebe --- /dev/null +++ b/ast/base.go @@ -0,0 +1,88 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ast + +import "github.com/pingcap/tidb/util/types" + +// node is the struct implements node interface except for Accept method. +// Node implementations should embed it in. +type node struct { + text string +} + +// SetText implements Node interface. +func (n *node) SetText(text string) { + n.text = text +} + +// Text implements Node interface. +func (n *node) Text() string { + return n.text +} + +// stmtNode implements StmtNode interface. +// Statement implementations should embed it in. +type stmtNode struct { + node +} + +// statement implements StmtNode interface. +func (sn *stmtNode) statement() {} + +// ddlNode implements DDLNode interface. +// DDL implementations should embed it in. +type ddlNode struct { + stmtNode +} + +// ddlStatement implements DDLNode interface. +func (dn *ddlNode) ddlStatement() {} + +// dmlNode is the struct implements DMLNode interface. +// DML implementations should embed it in. +type dmlNode struct { + stmtNode +} + +// dmlStatement implements DMLNode interface. +func (dn *dmlNode) dmlStatement() {} + +// expressionNode is the struct implements Expression interface. +// Expression implementations should embed it in. +type exprNode struct { + node + tp *types.FieldType +} + +// IsStatic implements Expression interface. +func (en *exprNode) IsStatic() bool { + return false +} + +// SetType implements Expression interface. +func (en *exprNode) SetType(tp *types.FieldType) { + en.tp = tp +} + +// GetType implements Expression interface. +func (en *exprNode) GetType() *types.FieldType { + return en.tp +} + +type funcNode struct { + exprNode +} + +// FunctionExpression implements FounctionNode interface. +func (fn *funcNode) functionExpression() {} diff --git a/ast/ddl.go b/ast/ddl.go new file mode 100644 index 0000000000..a9bc7b28bd --- /dev/null +++ b/ast/ddl.go @@ -0,0 +1,525 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ast + +import ( + "github.com/pingcap/tidb/util/types" +) + +var ( + _ DDLNode = &CreateDatabaseStmt{} + _ DDLNode = &DropDatabaseStmt{} + _ DDLNode = &CreateTableStmt{} + _ DDLNode = &DropTableStmt{} + _ DDLNode = &CreateIndexStmt{} + _ DDLNode = &DropTableStmt{} + _ DDLNode = &AlterTableStmt{} + _ DDLNode = &TruncateTableStmt{} + _ Node = &IndexColName{} + _ Node = &ReferenceDef{} + _ Node = &ColumnOption{} + _ Node = &Constraint{} + _ Node = &ColumnDef{} + _ Node = &ColumnPosition{} + _ Node = &AlterTableSpec{} +) + +// CharsetOpt is used for parsing charset option from SQL. +type CharsetOpt struct { + Chs string + Col string +} + +// CreateDatabaseStmt is a statement to create a database. +// See: https://dev.mysql.com/doc/refman/5.7/en/create-database.html +type CreateDatabaseStmt struct { + ddlNode + + IfNotExists bool + Name string + Opt *CharsetOpt +} + +// Accept implements Node Accept interface. +func (cd *CreateDatabaseStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(cd) { + return cd, false + } + return v.Leave(cd) +} + +// DropDatabaseStmt is a statement to drop a database and all tables in the database. +// See: https://dev.mysql.com/doc/refman/5.7/en/drop-database.html +type DropDatabaseStmt struct { + ddlNode + + IfExists bool + Name string +} + +// Accept implements Node Accept interface. +func (dd *DropDatabaseStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(dd) { + return dd, false + } + return v.Leave(dd) +} + +// IndexColName is used for parsing index column name from SQL. +type IndexColName struct { + node + + Column *ColumnRefExpr + Length int +} + +// Accept implements Node Accept interface. +func (ic *IndexColName) Accept(v Visitor) (Node, bool) { + if !v.Enter(ic) { + return ic, false + } + node, ok := ic.Column.Accept(v) + if !ok { + return ic, false + } + ic.Column = node.(*ColumnRefExpr) + return v.Leave(ic) +} + +// ReferenceDef is used for parsing foreign key reference option from SQL. +// See: http://dev.mysql.com/doc/refman/5.7/en/create-table-foreign-keys.html +type ReferenceDef struct { + node + + Table *TableRef + IndexColNames []*IndexColName +} + +// Accept implements Node Accept interface. +func (rd *ReferenceDef) Accept(v Visitor) (Node, bool) { + if !v.Enter(rd) { + return rd, false + } + node, ok := rd.Table.Accept(v) + if !ok { + return rd, false + } + rd.Table = node.(*TableRef) + for i, val := range rd.IndexColNames { + node, ok = val.Accept(v) + if !ok { + return rd, false + } + rd.IndexColNames[i] = node.(*IndexColName) + } + return v.Leave(rd) +} + +// ColumnOptionType is the type for ColumnOption. +type ColumnOptionType int + +// ColumnOption types. +const ( + ColumnOptionNoOption ColumnOptionType = iota + ColumnOptionPrimaryKey + ColumnOptionNotNull + ColumnOptionAutoIncrement + ColumnOptionDefaultValue + ColumnOptionUniq + ColumnOptionIndex + ColumnOptionUniqIndex + ColumnOptionKey + ColumnOptionUniqKey + ColumnOptionNull + ColumnOptionOnUpdate // For Timestamp and Datetime only. + ColumnOptionFulltext + ColumnOptionComment +) + +// ColumnOption is used for parsing column constraint info from SQL. +type ColumnOption struct { + node + + Tp ColumnOptionType + // The value For Default or On Update. + Val ExprNode +} + +// Accept implements Node Accept interface. +func (co *ColumnOption) Accept(v Visitor) (Node, bool) { + if !v.Enter(co) { + return co, false + } + if co.Val != nil { + node, ok := co.Val.Accept(v) + if !ok { + return co, false + } + co.Val = node.(ExprNode) + } + return v.Leave(co) +} + +// ConstraintType is the type for Constraint. +type ConstraintType int + +// ConstraintTypes +const ( + ConstraintNoConstraint ConstraintType = iota + ConstraintPrimaryKey + ConstraintKey + ConstraintIndex + ConstraintUniq + ConstraintUniqKey + ConstraintUniqIndex + ConstraintForeignKey +) + +// Constraint is constraint for table definition. +type Constraint struct { + node + + Tp ConstraintType + Name string + + // Used for PRIMARY KEY, UNIQUE, ...... + Keys []*IndexColName + + // Used for foreign key. + Refer *ReferenceDef +} + +// Accept implements Node Accept interface. +func (tc *Constraint) Accept(v Visitor) (Node, bool) { + if !v.Enter(tc) { + return tc, false + } + for i, val := range tc.Keys { + node, ok := val.Accept(v) + if !ok { + return tc, false + } + tc.Keys[i] = node.(*IndexColName) + } + if tc.Refer != nil { + node, ok := tc.Refer.Accept(v) + if !ok { + return tc, false + } + tc.Refer = node.(*ReferenceDef) + } + return v.Leave(tc) +} + +// ColumnDef is used for parsing column definition from SQL. +type ColumnDef struct { + node + + Name string + Tp *types.FieldType + Options []*ColumnOption +} + +// Accept implements Node Accept interface. +func (cd *ColumnDef) Accept(v Visitor) (Node, bool) { + if !v.Enter(cd) { + return cd, false + } + for i, val := range cd.Options { + node, ok := val.Accept(v) + if !ok { + return cd, false + } + cd.Options[i] = node.(*ColumnOption) + } + return v.Leave(cd) +} + +// CreateTableStmt is a statement to create a table. +// See: https://dev.mysql.com/doc/refman/5.7/en/create-table.html +type CreateTableStmt struct { + ddlNode + + IfNotExists bool + Table *TableRef + Cols []*ColumnDef + Constraints []*Constraint + Options []*TableOption +} + +// Accept implements Node Accept interface. +func (ct *CreateTableStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(ct) { + return ct, false + } + node, ok := ct.Table.Accept(v) + if !ok { + return ct, false + } + ct.Table = node.(*TableRef) + for i, val := range ct.Cols { + node, ok = val.Accept(v) + if !ok { + return ct, false + } + ct.Cols[i] = node.(*ColumnDef) + } + for i, val := range ct.Constraints { + node, ok = val.Accept(v) + if !ok { + return ct, false + } + ct.Constraints[i] = node.(*Constraint) + } + return v.Leave(ct) +} + +// DropTableStmt is a statement to drop one or more tables. +// See: https://dev.mysql.com/doc/refman/5.7/en/drop-table.html +type DropTableStmt struct { + ddlNode + + IfExists bool + TableRefs []*TableRef +} + +// Accept implements Node Accept interface. +func (dt *DropTableStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(dt) { + return dt, false + } + for i, val := range dt.TableRefs { + node, ok := val.Accept(v) + if !ok { + return dt, false + } + dt.TableRefs[i] = node.(*TableRef) + } + return v.Leave(dt) +} + +// CreateIndexStmt is a statement to create an index. +// See: https://dev.mysql.com/doc/refman/5.7/en/create-index.html +type CreateIndexStmt struct { + ddlNode + + IndexName string + Table *TableRef + Unique bool + IndexColNames []*IndexColName +} + +// Accept implements Node Accept interface. +func (ci *CreateIndexStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(ci) { + return ci, false + } + node, ok := ci.Table.Accept(v) + if !ok { + return ci, false + } + ci.Table = node.(*TableRef) + for i, val := range ci.IndexColNames { + node, ok = val.Accept(v) + if !ok { + return ci, false + } + ci.IndexColNames[i] = node.(*IndexColName) + } + return v.Leave(ci) +} + +// DropIndexStmt is a statement to drop the index. +// See: https://dev.mysql.com/doc/refman/5.7/en/drop-index.html +type DropIndexStmt struct { + ddlNode + + IfExists bool + IndexName string +} + +// Accept implements Node Accept interface. +func (di *DropIndexStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(di) { + return di, false + } + return v.Leave(di) +} + +// TableOptionType is the type for TableOption +type TableOptionType int + +// TableOption types. +const ( + TableOptionNone TableOptionType = iota + TableOptionEngine + TableOptionCharset + TableOptionCollate + TableOptionAutoIncrement + TableOptionComment + TableOptionAvgRowLength + TableOptionCheckSum + TableOptionCompression + TableOptionConnection + TableOptionPassword + TableOptionKeyBlockSize + TableOptionMaxRows + TableOptionMinRows +) + +// TableOption is used for parsing table option from SQL. +type TableOption struct { + Tp TableOptionType + StrValue string + UintValue uint64 +} + +// ColumnPositionType is the type for ColumnPosition. +type ColumnPositionType int + +// ColumnPosition Types +const ( + ColumnPositionNone ColumnPositionType = iota + ColumnPositionFirst + ColumnPositionAfter +) + +// ColumnPosition represent the position of the newly added column +type ColumnPosition struct { + node + // ColumnPositionNone | ColumnPositionFirst | ColumnPositionAfter + Tp ColumnPositionType + // RelativeColumn is the column the newly added column after if type is ColumnPositionAfter + RelativeColumn *ColumnRefExpr +} + +// Accept implements Node Accept interface. +func (cp *ColumnPosition) Accept(v Visitor) (Node, bool) { + if !v.Enter(cp) { + return cp, false + } + node, ok := cp.RelativeColumn.Accept(v) + if !ok { + return cp, false + } + cp.RelativeColumn = node.(*ColumnRefExpr) + return v.Leave(cp) +} + +// AlterTableType is the type for AlterTableSpec. +type AlterTableType int + +// AlterTable types. +const ( + AlterTableOption AlterTableType = iota + 1 + AlterTableAddColumn + AlterTableAddConstraint + AlterTableDropColumn + AlterTableDropPrimaryKey + AlterTableDropIndex + AlterTableDropForeignKey + +// TODO: Add more actions +) + +// AlterTableSpec represents alter table specification. +type AlterTableSpec struct { + node + + Tp AlterTableType + Name string + Constraint *Constraint + TableOpts []*TableOption + Column *ColumnDef + Position *ColumnPosition +} + +// Accept implements Node Accept interface. +func (as *AlterTableSpec) Accept(v Visitor) (Node, bool) { + if !v.Enter(as) { + return as, false + } + if as.Constraint != nil { + node, ok := as.Constraint.Accept(v) + if !ok { + return as, false + } + as.Constraint = node.(*Constraint) + } + if as.Column != nil { + node, ok := as.Column.Accept(v) + if !ok { + return as, false + } + as.Column = node.(*ColumnDef) + } + if as.Position != nil { + node, ok := as.Position.Accept(v) + if !ok { + return as, false + } + as.Position = node.(*ColumnPosition) + } + return v.Leave(as) +} + +// AlterTableStmt is a statement to change the structure of a table. +// See: https://dev.mysql.com/doc/refman/5.7/en/alter-table.html +type AlterTableStmt struct { + ddlNode + + Table *TableRef + Specs []*AlterTableSpec +} + +// Accept implements Node Accept interface. +func (at *AlterTableStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(at) { + return at, false + } + node, ok := at.Table.Accept(v) + if !ok { + return at, false + } + at.Table = node.(*TableRef) + for i, val := range at.Specs { + node, ok = val.Accept(v) + if !ok { + return at, false + } + at.Specs[i] = node.(*AlterTableSpec) + } + return v.Leave(at) +} + +// TruncateTableStmt is a statement to empty a table completely. +// See: https://dev.mysql.com/doc/refman/5.7/en/truncate-table.html +type TruncateTableStmt struct { + ddlNode + + Table *TableRef +} + +// Accept implements Node Accept interface. +func (ts *TruncateTableStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(ts) { + return ts, false + } + node, ok := ts.Table.Accept(v) + if !ok { + return ts, false + } + ts.Table = node.(*TableRef) + return v.Leave(ts) +} diff --git a/ast/dml.go b/ast/dml.go new file mode 100644 index 0000000000..9c1074cb13 --- /dev/null +++ b/ast/dml.go @@ -0,0 +1,409 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ast + +import ( + "github.com/pingcap/tidb/model" +) + +var ( + _ DMLNode = &InsertStmt{} + _ DMLNode = &DeleteStmt{} + _ DMLNode = &UpdateStmt{} + _ DMLNode = &SelectStmt{} + _ Node = &Join{} + _ Node = &Union{} + _ Node = &TableRef{} + _ Node = &TableSource{} + _ Node = &Assignment{} +) + +// JoinType is join type, including cross/left/right/full. +type JoinType int + +const ( + // CrossJoin is cross join type. + CrossJoin JoinType = iota + 1 + // LeftJoin is left Join type. + LeftJoin + // RightJoin is right Join type. + RightJoin +) + +// Join represents table join. +type Join struct { + node + + // Left table can be TableSource or JoinNode. + Left Node + // Right table can be TableSource or JoinNode or nil. + Right Node + // Tp represents join type. + Tp JoinType +} + +// Accept implements Node Accept interface. +func (j *Join) Accept(v Visitor) (Node, bool) { + if !v.Enter(j) { + return j, false + } + node, ok := j.Left.Accept(v) + if !ok { + return j, false + } + j.Left = node + if j.Right != nil { + node, ok = j.Right.Accept(v) + if !ok { + return j, false + } + j.Right = node + } + return v.Leave(j) +} + +// TableRef represents a reference to actual table. +type TableRef struct { + node + + Schema model.CIStr + Name model.CIStr +} + +// Accept implements Node Accept interface. +func (tr *TableRef) Accept(v Visitor) (Node, bool) { + if !v.Enter(tr) { + return tr, false + } + return v.Leave(tr) +} + +// TableSource represents table source with a name. +type TableSource struct { + node + + // Source is the source of the data, can be a TableRef, + // a SubQuery, or a JoinNode. + Source Node + + // Name is the alias name of the table source. + Name string +} + +// Accept implements Node Accept interface. +func (ts *TableSource) Accept(v Visitor) (Node, bool) { + if !v.Enter(ts) { + return ts, false + } + node, ok := ts.Source.Accept(v) + if !ok { + return ts, false + } + ts.Source = node + return v.Leave(ts) +} + +// Union represents union select statement. +type Union struct { + node + + Select *SelectStmt +} + +// Accept implements Node Accept interface. +func (u *Union) Accept(v Visitor) (Node, bool) { + if !v.Enter(u) { + return u, false + } + node, ok := u.Select.Accept(v) + if !ok { + return u, false + } + u.Select = node.(*SelectStmt) + return v.Leave(u) +} + +// SelectLockType is the lock type for SelectStmt. +type SelectLockType int + +// Select lock types. +const ( + SelectLockNone SelectLockType = iota + SelectLockForUpdate + SelectLockInShareMode +) + +// SelectStmt represents the select query node. +type SelectStmt struct { + dmlNode + + // Distinct represents if the select has distinct option. + Distinct bool + // Fields is the select expression list. + Fields []ExprNode + // From is the from clause of the query. + From *Join + // Where is the where clause in select statement. + Where ExprNode + // GroupBy is the group by expression list. + GroupBy []ExprNode + // Having is the having condition. + Having ExprNode + // OrderBy is the odering expression list. + OrderBy []ExprNode + // Offset is the offset value. + Offset int + // Limit is the limit value. + Limit int + // Lock is the lock type + LockTp SelectLockType + // Unions is the union select statement. + Unions []*Union +} + +// Accept implements Node Accept interface. +func (sn *SelectStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(sn) { + return sn, false + } + for i, val := range sn.Fields { + node, ok := val.Accept(v) + if !ok { + return sn, false + } + sn.Fields[i] = node.(ExprNode) + } + if sn.From != nil { + node, ok := sn.From.Accept(v) + if !ok { + return sn, false + } + sn.From = node.(*Join) + } + + if sn.Where != nil { + node, ok := sn.Where.Accept(v) + if !ok { + return sn, false + } + sn.Where = node.(ExprNode) + } + + for i, val := range sn.GroupBy { + node, ok := val.Accept(v) + if !ok { + return sn, false + } + sn.GroupBy[i] = node.(ExprNode) + } + if sn.Having != nil { + node, ok := sn.Having.Accept(v) + if !ok { + return sn, false + } + sn.Having = node.(ExprNode) + } + + for i, val := range sn.OrderBy { + node, ok := val.Accept(v) + if !ok { + return sn, false + } + sn.OrderBy[i] = node.(ExprNode) + } + + for i, val := range sn.Unions { + node, ok := val.Accept(v) + if !ok { + return sn, false + } + sn.Unions[i] = node.(*Union) + } + return v.Leave(sn) +} + +// Assignment is the expression for assignment, like a = 1. +type Assignment struct { + node + // Column is the column reference to be assigned. + Column *ColumnRefExpr + // Expr is the expression assigning to ColName. + Expr ExprNode +} + +// Accept implements Node Accept interface. +func (as *Assignment) Accept(v Visitor) (Node, bool) { + if !v.Enter(as) { + return as, false + } + node, ok := as.Column.Accept(v) + if !ok { + return as, false + } + as.Column = node.(*ColumnRefExpr) + node, ok = as.Expr.Accept(v) + if !ok { + return as, false + } + as.Expr = node.(ExprNode) + return v.Leave(as) +} + +// InsertStmt is a statement to insert new rows into an existing table. +// See: https://dev.mysql.com/doc/refman/5.7/en/insert.html +type InsertStmt struct { + dmlNode + + Columns []*ColumnRefExpr + Lists [][]ExprNode + Table *TableRef + Setlist []*Assignment + Priority int + OnDuplicate []*Assignment +} + +// Accept implements Node Accept interface. +func (in *InsertStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(in) { + return in, false + } + for i, val := range in.Columns { + node, ok := val.Accept(v) + if !ok { + return in, false + } + in.Columns[i] = node.(*ColumnRefExpr) + } + for i, list := range in.Lists { + for j, val := range list { + node, ok := val.Accept(v) + if !ok { + return in, false + } + in.Lists[i][j] = node.(ExprNode) + } + } + for i, val := range in.Setlist { + node, ok := val.Accept(v) + if !ok { + return in, false + } + in.Setlist[i] = node.(*Assignment) + } + for i, val := range in.OnDuplicate { + node, ok := val.Accept(v) + if !ok { + return in, false + } + in.OnDuplicate[i] = node.(*Assignment) + } + return v.Leave(in) +} + +// DeleteStmt is a statement to delete rows from table. +// See: https://dev.mysql.com/doc/refman/5.7/en/delete.html +type DeleteStmt struct { + dmlNode + + Tables []*TableRef + Where ExprNode + Order []ExprNode + Limit int + LowPriority bool + Ignore bool + Quick bool + MultiTable bool + BeforeFrom bool +} + +// Accept implements Node Accept interface. +func (de *DeleteStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(de) { + return de, false + } + for i, val := range de.Tables { + node, ok := val.Accept(v) + if !ok { + return de, false + } + de.Tables[i] = node.(*TableRef) + } + + if de.Where != nil { + node, ok := de.Where.Accept(v) + if !ok { + return de, false + } + de.Where = node.(ExprNode) + } + + for i, val := range de.Order { + node, ok := val.Accept(v) + if !ok { + return de, false + } + de.Order[i] = node.(ExprNode) + } + return v.Leave(de) +} + +// UpdateStmt is a statement to update columns of existing rows in tables with new values. +// See: https://dev.mysql.com/doc/refman/5.7/en/update.html +type UpdateStmt struct { + dmlNode + + TableRefs *Join + List []*Assignment + Where ExprNode + Order []ExprNode + Limit int + LowPriority bool + Ignore bool + MultipleTable bool +} + +// Accept implements Node Accept interface. +func (up *UpdateStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(up) { + return up, false + } + node, ok := up.TableRefs.Accept(v) + if !ok { + return up, false + } + up.TableRefs = node.(*Join) + for i, val := range up.List { + node, ok = val.Accept(v) + if !ok { + return up, false + } + up.List[i] = node.(*Assignment) + } + if up.Where != nil { + node, ok = up.Where.Accept(v) + if !ok { + return up, false + } + up.Where = node.(ExprNode) + } + + for i, val := range up.Order { + node, ok = val.Accept(v) + if !ok { + return up, false + } + up.Order[i] = node.(ExprNode) + } + return v.Leave(up) +} diff --git a/ast/expressions.go b/ast/expressions.go new file mode 100644 index 0000000000..d717561cde --- /dev/null +++ b/ast/expressions.go @@ -0,0 +1,669 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ast + +import ( + "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/parser/opcode" +) + +var ( + _ ExprNode = &ValueExpr{} + _ ExprNode = &BetweenExpr{} + _ ExprNode = &BinaryOperationExpr{} + _ Node = &WhenClause{} + _ ExprNode = &CaseExpr{} + _ ExprNode = &SubqueryExpr{} + _ ExprNode = &CompareSubqueryExpr{} + _ ExprNode = &ColumnRefExpr{} + _ ExprNode = &DefaultExpr{} + _ ExprNode = &ExistsSubqueryExpr{} + _ ExprNode = &PatternInExpr{} + _ ExprNode = &IsNullExpr{} + _ ExprNode = &IsTruthExpr{} + _ ExprNode = &PatternLikeExpr{} + _ ExprNode = &ParamMarkerExpr{} + _ ExprNode = &ParenthesesExpr{} + _ ExprNode = &PositionExpr{} + _ ExprNode = &PatternRegexpExpr{} + _ ExprNode = &RowExpr{} + _ ExprNode = &UnaryOperationExpr{} + _ ExprNode = &ValuesExpr{} + _ ExprNode = &VariableExpr{} +) + +// ValueExpr is the simple value expression. +type ValueExpr struct { + exprNode + // Val is the literal value. + Val interface{} +} + +// IsStatic implements ExprNode interface. +func (val *ValueExpr) IsStatic() bool { + return true +} + +// Accept implements Node interface. +func (val *ValueExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(val) { + return val, false + } + return v.Leave(val) +} + +// BetweenExpr is for "between and" or "not between and" expression. +type BetweenExpr struct { + exprNode + // Expr is the expression to be checked. + Expr ExprNode + // Left is the expression for minimal value in the range. + Left ExprNode + // Right is the expression for maximum value in the range. + Right ExprNode + // Not is true, the expression is "not between and". + Not bool +} + +// Accept implements Node interface. +func (b *BetweenExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(b) { + return b, false + } + + node, ok := b.Expr.Accept(v) + if !ok { + return b, false + } + b.Expr = node.(ExprNode) + + node, ok = b.Left.Accept(v) + if !ok { + return b, false + } + b.Left = node.(ExprNode) + + node, ok = b.Right.Accept(v) + if !ok { + return b, false + } + b.Right = node.(ExprNode) + + return v.Leave(b) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (b *BetweenExpr) IsStatic() bool { + return b.Expr.IsStatic() && b.Left.IsStatic() && b.Right.IsStatic() +} + +// BinaryOperationExpr is for binary operation like 1 + 1, 1 - 1, etc. +type BinaryOperationExpr struct { + exprNode + // Op is the operator code for BinaryOperation. + Op opcode.Op + // L is the left expression in BinaryOperation. + L ExprNode + // R is the right expression in BinaryOperation. + R ExprNode +} + +// Accept implements Node interface. +func (o *BinaryOperationExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(o) { + return o, false + } + + node, ok := o.L.Accept(v) + if !ok { + return o, false + } + o.L = node.(ExprNode) + + node, ok = o.R.Accept(v) + if !ok { + return o, false + } + o.R = node.(ExprNode) + + return v.Leave(o) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (o *BinaryOperationExpr) IsStatic() bool { + return o.L.IsStatic() && o.R.IsStatic() +} + +// WhenClause is the when clause in Case expression for "when condition then result". +type WhenClause struct { + node + // Expr is the condition expression in WhenClause. + Expr ExprNode + // Result is the result expression in WhenClause. + Result ExprNode +} + +// Accept implements Node Accept interface. +func (w *WhenClause) Accept(v Visitor) (Node, bool) { + if !v.Enter(w) { + return w, false + } + node, ok := w.Expr.Accept(v) + if !ok { + return w, false + } + w.Expr = node.(ExprNode) + + node, ok = w.Result.Accept(v) + if !ok { + return w, false + } + w.Result = node.(ExprNode) + return v.Leave(w) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (w *WhenClause) IsStatic() bool { + return w.Expr.IsStatic() && w.Result.IsStatic() +} + +// CaseExpr is the case expression. +type CaseExpr struct { + exprNode + // Value is the compare value expression. + Value ExprNode + // WhenClauses is the condition check expression. + WhenClauses []*WhenClause + // ElseClause is the else result expression. + ElseClause ExprNode +} + +// Accept implements Node Accept interface. +func (f *CaseExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(f) { + return f, false + } + if f.Value != nil { + node, ok := f.Value.Accept(v) + if !ok { + return f, false + } + f.Value = node.(ExprNode) + } + for i, val := range f.WhenClauses { + node, ok := val.Accept(v) + if !ok { + return f, false + } + f.WhenClauses[i] = node.(*WhenClause) + } + if f.ElseClause != nil { + node, ok := f.ElseClause.Accept(v) + if !ok { + return f, false + } + f.ElseClause = node.(ExprNode) + } + return v.Leave(f) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (f *CaseExpr) IsStatic() bool { + if f.Value != nil && !f.Value.IsStatic() { + return false + } + for _, w := range f.WhenClauses { + if !w.IsStatic() { + return false + } + } + if f.ElseClause != nil && !f.ElseClause.IsStatic() { + return false + } + return true +} + +// SubqueryExpr represents a sub query. +type SubqueryExpr struct { + exprNode + // Query is the query SelectNode. + Query *SelectStmt +} + +// Accept implements Node Accept interface. +func (sq *SubqueryExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(sq) { + return sq, false + } + node, ok := sq.Query.Accept(v) + if !ok { + return sq, false + } + sq.Query = node.(*SelectStmt) + return v.Leave(sq) +} + +// CompareSubqueryExpr is the expression for "expr cmp (select ...)". +// See: https://dev.mysql.com/doc/refman/5.7/en/comparisons-using-subqueries.html +// See: https://dev.mysql.com/doc/refman/5.7/en/any-in-some-subqueries.html +// See: https://dev.mysql.com/doc/refman/5.7/en/all-subqueries.html +type CompareSubqueryExpr struct { + exprNode + // L is the left expression + L ExprNode + // Op is the comparison opcode. + Op opcode.Op + // R is the sub query for right expression. + R *SubqueryExpr + // All is true, we should compare all records in subquery. + All bool +} + +// Accept implements Node Accept interface. +func (cs *CompareSubqueryExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(cs) { + return cs, false + } + node, ok := cs.L.Accept(v) + if !ok { + return cs, false + } + cs.L = node.(ExprNode) + node, ok = cs.R.Accept(v) + if !ok { + return cs, false + } + cs.R = node.(*SubqueryExpr) + return v.Leave(cs) +} + +// ColumnRefExpr represents a column reference. +type ColumnRefExpr struct { + exprNode + + // Name is the referenced column name. + Name model.CIStr +} + +// Accept implements Node Accept interface. +func (cr *ColumnRefExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(cr) { + return cr, false + } + return v.Leave(cr) +} + +// DefaultExpr is the default expression using default value for a column. +type DefaultExpr struct { + exprNode + // Name is the column name. + Name string +} + +// Accept implements Node Accept interface. +func (d *DefaultExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(d) { + return d, false + } + return v.Leave(d) +} + +// ExistsSubqueryExpr is the expression for "exists (select ...)". +// https://dev.mysql.com/doc/refman/5.7/en/exists-and-not-exists-subqueries.html +type ExistsSubqueryExpr struct { + exprNode + // Sel is the sub query. + Sel *SubqueryExpr +} + +// Accept implements Node Accept interface. +func (es *ExistsSubqueryExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(es) { + return es, false + } + node, ok := es.Sel.Accept(v) + if !ok { + return es, false + } + es.Sel = node.(*SubqueryExpr) + return v.Leave(es) +} + +// PatternInExpr is the expression for in operator, like "expr in (1, 2, 3)" or "expr in (select c from t)". +type PatternInExpr struct { + exprNode + // Expr is the value expression to be compared. + Expr ExprNode + // List is the list expression in compare list. + List []ExprNode + // Not is true, the expression is "not in". + Not bool + // Sel is the sub query. + Sel *SubqueryExpr +} + +// Accept implements Node Accept interface. +func (pi *PatternInExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(pi) { + return pi, false + } + node, ok := pi.Expr.Accept(v) + if !ok { + return pi, false + } + pi.Expr = node.(ExprNode) + for i, val := range pi.List { + node, ok = val.Accept(v) + if !ok { + return pi, false + } + pi.List[i] = node.(ExprNode) + } + if pi.Sel != nil { + node, ok = pi.Sel.Accept(v) + if !ok { + return pi, false + } + pi.Sel = node.(*SubqueryExpr) + } + return v.Leave(pi) +} + +// IsNullExpr is the expression for null check. +type IsNullExpr struct { + exprNode + // Expr is the expression to be checked. + Expr ExprNode + // Not is true, the expression is "is not null". + Not bool +} + +// Accept implements Node Accept interface. +func (is *IsNullExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(is) { + return is, false + } + node, ok := is.Expr.Accept(v) + if !ok { + return is, false + } + is.Expr = node.(ExprNode) + return v.Leave(is) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (is *IsNullExpr) IsStatic() bool { + return is.Expr.IsStatic() +} + +// IsTruthExpr is the expression for true/false check. +type IsTruthExpr struct { + exprNode + // Expr is the expression to be checked. + Expr ExprNode + // Not is true, the expression is "is not true/false". + Not bool + // True indicates checking true or false. + True int64 +} + +// Accept implements Node Accept interface. +func (is *IsTruthExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(is) { + return is, false + } + node, ok := is.Expr.Accept(v) + if !ok { + return is, false + } + is.Expr = node.(ExprNode) + return v.Leave(is) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (is *IsTruthExpr) IsStatic() bool { + return is.Expr.IsStatic() +} + +// PatternLikeExpr is the expression for like operator, e.g, expr like "%123%" +type PatternLikeExpr struct { + exprNode + // Expr is the expression to be checked. + Expr ExprNode + // Pattern is the like expression. + Pattern ExprNode + // Not is true, the expression is "not like". + Not bool +} + +// Accept implements Node Accept interface. +func (pl *PatternLikeExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(pl) { + return pl, false + } + node, ok := pl.Expr.Accept(v) + if !ok { + return pl, false + } + pl.Expr = node.(ExprNode) + node, ok = pl.Pattern.Accept(v) + if !ok { + return pl, false + } + pl.Pattern = node.(ExprNode) + return v.Leave(pl) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (pl *PatternLikeExpr) IsStatic() bool { + return pl.Expr.IsStatic() && pl.Pattern.IsStatic() +} + +// ParamMarkerExpr expresion holds a place for another expression. +// Used in parsing prepare statement. +type ParamMarkerExpr struct { + exprNode +} + +// Accept implements Node Accept interface. +func (pm *ParamMarkerExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(pm) { + return pm, false + } + return v.Leave(pm) +} + +// ParenthesesExpr is the parentheses expression. +type ParenthesesExpr struct { + exprNode + // Expr is the expression in parentheses. + Expr ExprNode +} + +// Accept implements Node Accept interface. +func (p *ParenthesesExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(p) { + return p, false + } + if p.Expr != nil { + node, ok := p.Expr.Accept(v) + if !ok { + return p, false + } + p.Expr = node.(ExprNode) + } + return v.Leave(p) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (p *ParenthesesExpr) IsStatic() bool { + return p.Expr.IsStatic() +} + +// PositionExpr is the expression for order by and group by position. +// MySQL use position expression started from 1, it looks a little confused inner. +// maybe later we will use 0 at first. +type PositionExpr struct { + exprNode + // N is the position, started from 1 now. + N int + // Name is the corresponding field name if we want better format and explain instead of position. + Name string +} + +// IsStatic implements the ExprNode IsStatic interface. +func (p *PositionExpr) IsStatic() bool { + return true +} + +// Accept implements Node Accept interface. +func (p *PositionExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(p) { + return p, false + } + return v.Leave(p) +} + +// PatternRegexpExpr is the pattern expression for pattern match. +type PatternRegexpExpr struct { + exprNode + // Expr is the expression to be checked. + Expr ExprNode + // Pattern is the expression for pattern. + Pattern ExprNode + // Not is true, the expression is "not rlike", + Not bool +} + +// Accept implements Node Accept interface. +func (p *PatternRegexpExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(p) { + return p, false + } + node, ok := p.Expr.Accept(v) + if !ok { + return p, false + } + p.Expr = node.(ExprNode) + node, ok = p.Pattern.Accept(v) + if !ok { + return p, false + } + p.Pattern = node.(ExprNode) + return v.Leave(p) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (p *PatternRegexpExpr) IsStatic() bool { + return p.Expr.IsStatic() && p.Pattern.IsStatic() +} + +// RowExpr is the expression for row constructor. +// See https://dev.mysql.com/doc/refman/5.7/en/row-subqueries.html +type RowExpr struct { + exprNode + + Values []ExprNode +} + +// Accept implements Node Accept interface. +func (r *RowExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(r) { + return r, false + } + for i, val := range r.Values { + node, ok := val.Accept(v) + if !ok { + return r, false + } + r.Values[i] = node.(ExprNode) + } + return v.Leave(r) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (r *RowExpr) IsStatic() bool { + for _, v := range r.Values { + if !v.IsStatic() { + return false + } + } + return true +} + +// UnaryOperationExpr is the expression for unary operator. +type UnaryOperationExpr struct { + exprNode + // Op is the operator opcode. + Op opcode.Op + // V is the unary expression. + V ExprNode +} + +// Accept implements Node Accept interface. +func (u *UnaryOperationExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(u) { + return u, false + } + node, ok := u.V.Accept(v) + if !ok { + return u, false + } + u.V = node.(ExprNode) + return v.Leave(u) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (u *UnaryOperationExpr) IsStatic() bool { + return u.V.IsStatic() +} + +// ValuesExpr is the expression used in INSERT VALUES +type ValuesExpr struct { + exprNode + // model.CIStr is column name. + Column *ColumnRefExpr +} + +// Accept implements Node Accept interface. +func (va *ValuesExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(va) { + return va, false + } + node, ok := va.Column.Accept(v) + if !ok { + return va, false + } + va.Column = node.(*ColumnRefExpr) + return v.Leave(va) +} + +// VariableExpr is the expression for variable. +type VariableExpr struct { + exprNode + // Name is the variable name. + Name string + // IsGlobal indicates whether this variable is global. + IsGlobal bool + // IsSystem indicates whether this variable is a global variable in current session. + IsSystem bool +} + +// Accept implements Node Accept interface. +func (va *VariableExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(va) { + return va, false + } + return v.Leave(va) +} diff --git a/ast/functions.go b/ast/functions.go new file mode 100644 index 0000000000..95347c5d4f --- /dev/null +++ b/ast/functions.go @@ -0,0 +1,251 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ast + +import ( + "strings" + + "github.com/pingcap/tidb/expression/builtin" + "github.com/pingcap/tidb/util/types" +) + +var ( + _ FuncNode = &FuncCallExpr{} + _ FuncNode = &FuncExtractExpr{} + _ FuncNode = &FuncConvertExpr{} + _ FuncNode = &FuncCastExpr{} + _ FuncNode = &FuncSubstringExpr{} + _ FuncNode = &FuncTrimExpr{} +) + +// FuncCallExpr is for function expression. +type FuncCallExpr struct { + funcNode + // F is the function name. + F string + // Args is the function args. + Args []ExprNode + // Distinct only affetcts sum, avg, count, group_concat, + // so we can ignore it in other functions + Distinct bool +} + +// Accept implements Node interface. +func (c *FuncCallExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(c) { + return c, false + } + for i, val := range c.Args { + node, ok := val.Accept(v) + if !ok { + return c, false + } + c.Args[i] = node.(ExprNode) + } + return v.Leave(c) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (c *FuncCallExpr) IsStatic() bool { + v := builtin.Funcs[strings.ToLower(c.F)] + if v.F == nil || !v.IsStatic { + return false + } + + for _, v := range c.Args { + if !v.IsStatic() { + return false + } + } + return true +} + +// FuncExtractExpr is for time extract function. +// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_extract +type FuncExtractExpr struct { + funcNode + + Unit string + Date ExprNode +} + +// Accept implements Node Accept interface. +func (ex *FuncExtractExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(ex) { + return ex, false + } + node, ok := ex.Date.Accept(v) + if !ok { + return ex, false + } + ex.Date = node.(ExprNode) + return v.Leave(ex) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (ex *FuncExtractExpr) IsStatic() bool { + return ex.Date.IsStatic() +} + +// FuncConvertExpr provides a way to convert data between different character sets. +// See: https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html#function_convert +type FuncConvertExpr struct { + funcNode + // Expr is the expression to be converted. + Expr ExprNode + // Charset is the target character set to convert. + Charset string +} + +// IsStatic implements the ExprNode IsStatic interface. +func (f *FuncConvertExpr) IsStatic() bool { + return f.Expr.IsStatic() +} + +// Accept implements Node Accept interface. +func (f *FuncConvertExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(f) { + return f, false + } + node, ok := f.Expr.Accept(v) + if !ok { + return f, false + } + f.Expr = node.(ExprNode) + return v.Leave(f) +} + +// castOperatopr is the operator type for cast function. +type castFunctionType int + +// castFunction types +const ( + CastFunction castFunctionType = iota + 1 + ConvertFunction + BinaryOperator +) + +// FuncCastExpr is the cast function converting value to another type, e.g, cast(expr AS signed). +// See https://dev.mysql.com/doc/refman/5.7/en/cast-functions.html +type FuncCastExpr struct { + funcNode + // Expr is the expression to be converted. + Expr ExprNode + // Tp is the conversion type. + Tp *types.FieldType + // Cast, Convert and Binary share this struct. + FunctionType castFunctionType +} + +// IsStatic implements the ExprNode IsStatic interface. +func (f *FuncCastExpr) IsStatic() bool { + return f.Expr.IsStatic() +} + +// Accept implements Node Accept interface. +func (f *FuncCastExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(f) { + return f, false + } + node, ok := f.Expr.Accept(v) + if !ok { + return f, false + } + f.Expr = node.(ExprNode) + return v.Leave(f) +} + +// FuncSubstringExpr returns the substring as specified. +// See: https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_substring +type FuncSubstringExpr struct { + funcNode + + StrExpr ExprNode + Pos ExprNode + Len ExprNode +} + +// Accept implements Node Accept interface. +func (sf *FuncSubstringExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(sf) { + return sf, false + } + node, ok := sf.StrExpr.Accept(v) + if !ok { + return sf, false + } + sf.StrExpr = node.(ExprNode) + node, ok = sf.Pos.Accept(v) + if !ok { + return sf, false + } + sf.Pos = node.(ExprNode) + node, ok = sf.Len.Accept(v) + if !ok { + return sf, false + } + sf.Len = node.(ExprNode) + return v.Leave(sf) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (sf *FuncSubstringExpr) IsStatic() bool { + return sf.StrExpr.IsStatic() && sf.Pos.IsStatic() && sf.Len.IsStatic() +} + +type trimDirectionType int + +const ( + // TrimBothDefault trims from both direction by default. + TrimBothDefault trimDirectionType = iota + // TrimBoth trims from both direction with explicit notation. + TrimBoth + // TrimLeading trims from left. + TrimLeading + // TrimTrailing trims from right. + TrimTrailing +) + +// FuncTrimExpr remove leading/trailing/both remstr. +// See: https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim +type FuncTrimExpr struct { + funcNode + + Str ExprNode + RemStr ExprNode + Direction trimDirectionType +} + +// Accept implements Node Accept interface. +func (tf *FuncTrimExpr) Accept(v Visitor) (Node, bool) { + if !v.Enter(tf) { + return tf, false + } + node, ok := tf.Str.Accept(v) + if !ok { + return tf, false + } + tf.Str = node.(ExprNode) + node, ok = tf.RemStr.Accept(v) + if !ok { + return tf, false + } + tf.RemStr = node.(ExprNode) + return v.Leave(tf) +} + +// IsStatic implements the ExprNode IsStatic interface. +func (tf *FuncTrimExpr) IsStatic() bool { + return tf.Str.IsStatic() && tf.RemStr.IsStatic() +} diff --git a/ast/misc.go b/ast/misc.go new file mode 100644 index 0000000000..1e6d0feade --- /dev/null +++ b/ast/misc.go @@ -0,0 +1,283 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package ast + +var ( + _ StmtNode = &ExplainStmt{} + _ StmtNode = &PrepareStmt{} + _ StmtNode = &DeallocateStmt{} + _ StmtNode = &ExecuteStmt{} + _ StmtNode = &ShowStmt{} + _ StmtNode = &BeginStmt{} + _ StmtNode = &CommitStmt{} + _ StmtNode = &RollbackStmt{} + _ StmtNode = &UseStmt{} + _ StmtNode = &SetStmt{} + + _ Node = &VariableAssignment{} +) + +// AuthOption is used for parsing create use statement. +type AuthOption struct { + // AuthString/HashString can be empty, so we need to decide which one to use. + ByAuthString bool + AuthString string + HashString string + // TODO: support auth_plugin +} + +// ExplainStmt is a statement to provide information about how is SQL statement executed +// or get columns information in a table. +// See: https://dev.mysql.com/doc/refman/5.7/en/explain.html +type ExplainStmt struct { + stmtNode + + Stmt DMLNode +} + +// Accept implements Node Accept interface. +func (es *ExplainStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(es) { + return es, false + } + node, ok := es.Stmt.Accept(v) + if !ok { + return es, false + } + es.Stmt = node.(DMLNode) + return v.Leave(es) +} + +// PrepareStmt is a statement to prepares a SQL statement which contains placeholders, +// and it is executed with ExecuteStmt and released with DeallocateStmt. +// See: https://dev.mysql.com/doc/refman/5.7/en/prepare.html +type PrepareStmt struct { + stmtNode + + InPrepare bool // true for prepare mode, false for use mode + Name string + ID uint32 // For binary protocol, there is no Name but only ID + SQLStmt Node // The parsed statement from sql text with placeholder +} + +// Accept implements Node Accept interface. +func (ps *PrepareStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(ps) { + return ps, false + } + node, ok := ps.SQLStmt.Accept(v) + if !ok { + return ps, false + } + ps.SQLStmt = node + return v.Leave(ps) +} + +// DeallocateStmt is a statement to release PreparedStmt. +// See: https://dev.mysql.com/doc/refman/5.7/en/deallocate-prepare.html +type DeallocateStmt struct { + stmtNode + + Name string + ID uint32 // For binary protocol, there is no Name but only ID. +} + +// Accept implements Node Accept interface. +func (ds *DeallocateStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(ds) { + return ds, false + } + return v.Leave(ds) +} + +// ExecuteStmt is a statement to execute PreparedStmt. +// See: https://dev.mysql.com/doc/refman/5.7/en/execute.html +type ExecuteStmt struct { + stmtNode + + Name string + ID uint32 // For binary protocol, there is no Name but only ID + UsingVars []ExprNode +} + +// Accept implements Node Accept interface. +func (es *ExecuteStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(es) { + return es, false + } + for i, val := range es.UsingVars { + node, ok := val.Accept(v) + if !ok { + return es, false + } + es.UsingVars[i] = node.(ExprNode) + } + return v.Leave(es) +} + +// ShowStmt is a statement to provide information about databases, tables, columns and so on. +// See: https://dev.mysql.com/doc/refman/5.7/en/show.html +type ShowStmt struct { + stmtNode + + Target int // Databases/Tables/Columns/.... + DBName string + Table *TableRef // Used for showing columns. + Column *ColumnRefExpr // Used for `desc table column`. + Flag int // Some flag parsed from sql, such as FULL. + Full bool + + // Used by show variables + GlobalScope bool + Pattern *PatternLikeExpr + Where ExprNode +} + +// Accept implements Node Accept interface. +func (ss *ShowStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(ss) { + return ss, false + } + if ss.Table != nil { + node, ok := ss.Table.Accept(v) + if !ok { + return ss, false + } + ss.Table = node.(*TableRef) + } + if ss.Column != nil { + node, ok := ss.Column.Accept(v) + if !ok { + return ss, false + } + ss.Column = node.(*ColumnRefExpr) + } + if ss.Pattern != nil { + node, ok := ss.Pattern.Accept(v) + if !ok { + return ss, false + } + ss.Pattern = node.(*PatternLikeExpr) + } + if ss.Where != nil { + node, ok := ss.Where.Accept(v) + if !ok { + return ss, false + } + ss.Where = node.(ExprNode) + } + return v.Leave(ss) +} + +// BeginStmt is a statement to start a new transaction. +// See: https://dev.mysql.com/doc/refman/5.7/en/commit.html +type BeginStmt struct { + stmtNode +} + +// Accept implements Node Accept interface. +func (bs *BeginStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(bs) { + return bs, false + } + return v.Leave(bs) +} + +// CommitStmt is a statement to commit the current transaction. +// See: https://dev.mysql.com/doc/refman/5.7/en/commit.html +type CommitStmt struct { + stmtNode +} + +// Accept implements Node Accept interface. +func (cs *CommitStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(cs) { + return cs, false + } + return v.Leave(cs) +} + +// RollbackStmt is a statement to roll back the current transaction. +// See: https://dev.mysql.com/doc/refman/5.7/en/commit.html +type RollbackStmt struct { + stmtNode +} + +// Accept implements Node Accept interface. +func (rs *RollbackStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(rs) { + return rs, false + } + return v.Leave(rs) +} + +// UseStmt is a statement to use the DBName database as the current database. +// See: https://dev.mysql.com/doc/refman/5.7/en/use.html +type UseStmt struct { + stmtNode + + DBName string +} + +// Accept implements Node Accept interface. +func (us *UseStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(us) { + return us, false + } + return v.Leave(us) +} + +// VariableAssignment is a variable assignment struct. +type VariableAssignment struct { + node + Name string + Value ExprNode + IsGlobal bool + IsSystem bool +} + +// Accept implements Node interface. +func (va *VariableAssignment) Accept(v Visitor) (Node, bool) { + if !v.Enter(va) { + return va, false + } + node, ok := va.Value.Accept(v) + if !ok { + return va, false + } + va.Value = node.(ExprNode) + return v.Leave(va) +} + +// SetStmt is the statement to set variables. +type SetStmt struct { + stmtNode + // Variables is the list of variable assignment. + Variables []*VariableAssignment +} + +// Accept implements Node Accept interface. +func (set *SetStmt) Accept(v Visitor) (Node, bool) { + if !v.Enter(set) { + return set, false + } + for i, val := range set.Variables { + node, ok := val.Accept(v) + if !ok { + return set, false + } + set.Variables[i] = node.(*VariableAssignment) + } + return v.Leave(set) +} diff --git a/column/column.go b/column/column.go index 10f51297cb..6acdfe2c37 100644 --- a/column/column.go +++ b/column/column.go @@ -18,8 +18,6 @@ package column import ( - "bytes" - "fmt" "strings" "github.com/juju/errors" @@ -46,7 +44,7 @@ type IndexedCol struct { // String implements fmt.Stringer interface. func (c *Col) String() string { - ans := []string{c.Name.O, types.FieldTypeToStr(c.Tp, c.Charset)} + ans := []string{c.Name.O, types.TypeToStr(c.Tp, c.Charset)} if mysql.HasAutoIncrementFlag(c.Flag) { ans = append(ans, "AUTO_INCREMENT") } @@ -121,41 +119,11 @@ const defaultPrivileges string = "select,insert,update,references" // GetTypeDesc gets the description for column type. func (c *Col) GetTypeDesc() string { - var buf bytes.Buffer - - buf.WriteString(types.FieldTypeToStr(c.Tp, c.Charset)) - switch c.Tp { - case mysql.TypeSet, mysql.TypeEnum: - // Format is ENUM ('e1', 'e2') or SET ('e1', 'e2') - // If elem contain ', we will convert ' -> '' - elems := make([]string, len(c.Elems)) - for i := range elems { - elems[i] = strings.Replace(c.Elems[i], "'", "''", -1) - } - buf.WriteString(fmt.Sprintf("('%s')", strings.Join(elems, "','"))) - case mysql.TypeFloat, mysql.TypeDouble: - // if only float(M), we will use float. The same for double. - if c.Flen != -1 && c.Decimal != -1 { - buf.WriteString(fmt.Sprintf("(%d,%d)", c.Flen, c.Decimal)) - } - case mysql.TypeTimestamp, mysql.TypeDatetime, mysql.TypeDate: - if c.Decimal != -1 && c.Decimal != 0 { - buf.WriteString(fmt.Sprintf("(%d)", c.Decimal)) - } - default: - if c.Flen != -1 { - if c.Decimal == -1 { - buf.WriteString(fmt.Sprintf("(%d)", c.Flen)) - } else { - buf.WriteString(fmt.Sprintf("(%d,%d)", c.Flen, c.Decimal)) - } - } - } - + desc := c.FieldType.CompactStr() if mysql.HasUnsignedFlag(c.Flag) { - buf.WriteString(" UNSIGNED") + desc += " UNSIGNED" } - return buf.String() + return desc } // NewColDesc returns a new ColDesc for a column. diff --git a/ddl/ddl_test.go b/ddl/ddl_test.go index 7343b9674f..2945b69388 100644 --- a/ddl/ddl_test.go +++ b/ddl/ddl_test.go @@ -198,5 +198,5 @@ func statement(sql string) stmt.Statement { log.Debug("Compile", sql) lexer := parser.NewLexer(sql) parser.YYParse(lexer) - return lexer.Stmts()[0] + return lexer.Stmts()[0].(stmt.Statement) } diff --git a/expression/helper.go b/expression/helper.go index be573e3fba..69554c4354 100644 --- a/expression/helper.go +++ b/expression/helper.go @@ -104,11 +104,6 @@ func FastEval(v interface{}) interface{} { } } -// IsQualified returns whether name contains ".". -func IsQualified(name string) bool { - return strings.Contains(name, ".") -} - // Eval is a helper function evaluates expression v and do a panic if evaluating error. func Eval(v Expression, ctx context.Context, env map[interface{}]interface{}) (y interface{}) { var err error diff --git a/field/field_test.go b/field/field_test.go index cde7cb548f..c72d29b6e6 100644 --- a/field/field_test.go +++ b/field/field_test.go @@ -43,16 +43,16 @@ func (*testFieldSuite) TestField(c *C) { ft := types.NewFieldType(mysql.TypeLong) ft.Flen = 20 ft.Flag |= mysql.UnsignedFlag | mysql.ZerofillFlag - c.Assert(ft.String(), Equals, "int (20) UNSIGNED ZEROFILL") + c.Assert(ft.String(), Equals, "int(20) UNSIGNED ZEROFILL") ft = types.NewFieldType(mysql.TypeFloat) ft.Flen = 20 ft.Decimal = 10 - c.Assert(ft.String(), Equals, "float (20, 10)") + c.Assert(ft.String(), Equals, "float(20,10)") ft = types.NewFieldType(mysql.TypeTimestamp) ft.Decimal = 8 - c.Assert(ft.String(), Equals, "timestamp (8)") + c.Assert(ft.String(), Equals, "timestamp(8)") ft = types.NewFieldType(mysql.TypeVarchar) ft.Flag |= mysql.BinaryFlag diff --git a/field/result_field.go b/field/result_field.go index 9e3803b01a..2eb301b9d2 100644 --- a/field/result_field.go +++ b/field/result_field.go @@ -24,7 +24,6 @@ import ( "github.com/juju/errors" "github.com/pingcap/tidb/column" mysql "github.com/pingcap/tidb/mysqldef" - "github.com/pingcap/tidb/util/types" ) const ( @@ -89,14 +88,6 @@ func ColToResultField(col *column.Col, tableName string) *ResultField { TableName: tableName, OrgTableName: tableName, } - - if rf.Col.Flen == types.UnspecifiedLength { - rf.Col.Flen = 0 - } - if rf.Col.Decimal == types.UnspecifiedLength { - rf.Col.Decimal = 0 - } - // Keep things compatible for old clients. // Refer to mysql-server/sql/protocol.cc send_result_set_metadata() if rf.Tp == mysql.TypeVarchar { @@ -290,3 +281,8 @@ func CheckWildcardField(name string) (string, bool, error) { _, table, field := SplitQualifiedName(name) return table, field == "*", nil } + +// IsQualifiedName returns whether name contains "." or not. +func IsQualifiedName(name string) bool { + return strings.Contains(name, ".") +} diff --git a/kv/kv.go b/kv/kv.go index 90de6933ef..01223b61b2 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -25,7 +25,7 @@ type Key []byte // Next returns the next key in byte-order. func (k Key) Next() Key { - // add \x0 to the end of key + // add 0x0 to the end of key buf := make([]byte, len([]byte(k))+1) copy(buf, []byte(k)) return buf diff --git a/kv/memkv/btree_test.go b/kv/memkv/btree_test.go index 3527df0ddb..b36d52b108 100644 --- a/kv/memkv/btree_test.go +++ b/kv/memkv/btree_test.go @@ -32,13 +32,13 @@ func (*testBtreeSuite) TestBtree(c *C) { v := []interface{}{string(i + 1)} t.Set(k, v) } - for i := 0; i < 102400; i++ { + for i := 0; i < 1024; i++ { k := []interface{}{i} v := []interface{}{string(i)} t.Set(k, v) } // Delete - for i := 512; i < 102400; i++ { + for i := 512; i < 1024; i++ { k := []interface{}{i} t.Delete(k) } @@ -51,7 +51,7 @@ func (*testBtreeSuite) TestBtree(c *C) { c.Assert(v[0], Equals, string(i)) } // Get unexists key - for i := 512; i < 102400; i++ { + for i := 512; i < 1024; i++ { k := []interface{}{i} v, ok := t.Get(k) c.Assert(ok, IsFalse) diff --git a/kv/union_iter.go b/kv/union_iter.go index 3d3fa0f911..6b677c433c 100644 --- a/kv/union_iter.go +++ b/kv/union_iter.go @@ -117,7 +117,7 @@ func (iter *UnionIter) updateCur() { // Next implements the Iterator Next interface. func (iter *UnionIter) Next(f FnKeyCmp) (Iterator, error) { - if iter.curIsDirty == false { + if !iter.curIsDirty { iter.snapshotNext() } else { iter.dirtyNext() @@ -129,7 +129,7 @@ func (iter *UnionIter) Next(f FnKeyCmp) (Iterator, error) { // Value implements the Iterator Value interface. // Multi columns func (iter *UnionIter) Value() []byte { - if iter.curIsDirty == false { + if !iter.curIsDirty { return iter.snapshotIt.Value() } return iter.dirtyIt.Value() @@ -137,7 +137,7 @@ func (iter *UnionIter) Value() []byte { // Key implements the Iterator Key interface. func (iter *UnionIter) Key() string { - if iter.curIsDirty == false { + if !iter.curIsDirty { return string(DecodeKey([]byte(iter.snapshotIt.Key()))) } return string(DecodeKey(iter.dirtyIt.Key())) diff --git a/meta/meta.go b/meta/meta.go index 3f2ef0d59b..b0af58f6b5 100644 --- a/meta/meta.go +++ b/meta/meta.go @@ -15,6 +15,7 @@ package meta import ( "fmt" + "sync" "github.com/juju/errors" "github.com/ngaut/log" @@ -35,6 +36,10 @@ var ( globalIDKey = MakeMetaKey("mNextGlobalID") ) +var ( + globalIDMutex sync.Mutex +) + // MakeMetaKey creates meta key func MakeMetaKey(key string) []byte { return append([]byte{0x0}, key...) @@ -67,11 +72,13 @@ func AutoIDKey(tableID int64) string { if tableID == 0 { log.Error("Invalid tableID") } - return fmt.Sprintf("%s:%d_autoID", TableMetaPrefix, tableID) + return fmt.Sprintf("%s:%d_auto_id", TableMetaPrefix, tableID) } // GenGlobalID generates the next id in the store scope. func GenGlobalID(store kv.Storage) (ID int64, err error) { + globalIDMutex.Lock() + defer globalIDMutex.Unlock() err = kv.RunInNewTxn(store, true, func(txn kv.Transaction) error { ID, err = GenID(txn, globalIDKey, 1) if err != nil { diff --git a/meta/meta_test.go b/meta/meta_test.go index 316990dbf8..c7c758b254 100644 --- a/meta/meta_test.go +++ b/meta/meta_test.go @@ -54,9 +54,9 @@ func (*testSuite) TestT(c *C) { //For AutoIDKey mkey = []byte(meta.AutoIDKey(1)) - c.Assert(mkey, DeepEquals, meta.MakeMetaKey("mTable::1_autoID")) + c.Assert(mkey, DeepEquals, meta.MakeMetaKey("mTable::1_auto_id")) mkey = []byte(meta.AutoIDKey(0)) - c.Assert(mkey, DeepEquals, meta.MakeMetaKey("mTable::0_autoID")) + c.Assert(mkey, DeepEquals, meta.MakeMetaKey("mTable::0_auto_id")) // For GenGlobalID id, err = meta.GenGlobalID(store) diff --git a/mysqldef/errcode.go b/mysqldef/errcode.go index 4f40c4f66f..d486c36c90 100644 --- a/mysqldef/errcode.go +++ b/mysqldef/errcode.go @@ -16,870 +16,870 @@ package mysqldef // MySQL error code. // This value is numeric. It is not portable to other database systems. const ( - ErErrorFirst uint16 = 1000 - ErHashchk = 1000 - ErNisamchk = 1001 - ErNo = 1002 - ErYes = 1003 - ErCantCreateFile = 1004 - ErCantCreateTable = 1005 - ErCantCreateDb = 1006 - ErDbCreateExists = 1007 - ErDbDropExists = 1008 - ErDbDropDelete = 1009 - ErDbDropRmdir = 1010 - ErCantDeleteFile = 1011 - ErCantFindSystemRec = 1012 - ErCantGetStat = 1013 - ErCantGetWd = 1014 - ErCantLock = 1015 - ErCantOpenFile = 1016 - ErFileNotFound = 1017 - ErCantReadDir = 1018 - ErCantSetWd = 1019 - ErCheckread = 1020 - ErDiskFull = 1021 - ErDupKey = 1022 - ErErrorOnClose = 1023 - ErErrorOnRead = 1024 - ErErrorOnRename = 1025 - ErErrorOnWrite = 1026 - ErFileUsed = 1027 - ErFilsortAbort = 1028 - ErFormNotFound = 1029 - ErGetErrno = 1030 - ErIllegalHa = 1031 - ErKeyNotFound = 1032 - ErNotFormFile = 1033 - ErNotKeyfile = 1034 - ErOldKeyfile = 1035 - ErOpenAsReadonly = 1036 - ErOutofmemory = 1037 - ErOutOfSortmemory = 1038 - ErUnexpectedEOF = 1039 - ErConCountError = 1040 - ErOutOfResources = 1041 - ErBadHostError = 1042 - ErHandshakeError = 1043 - ErDbaccessDeniedError = 1044 - ErAccessDeniedError = 1045 - ErNoDbError = 1046 - ErUnknownComError = 1047 - ErBadNullError = 1048 - ErBadDbError = 1049 - ErTableExistsError = 1050 - ErBadTableError = 1051 - ErNonUniqError = 1052 - ErServerShutdown = 1053 - ErBadFieldError = 1054 - ErWrongFieldWithGroup = 1055 - ErWrongGroupField = 1056 - ErWrongSumSelect = 1057 - ErWrongValueCount = 1058 - ErTooLongIdent = 1059 - ErDupFieldname = 1060 - ErDupKeyname = 1061 - ErDupEntry = 1062 - ErWrongFieldSpec = 1063 - ErParseError = 1064 - ErEmptyQuery = 1065 - ErNonuniqTable = 1066 - ErInvalidDefault = 1067 - ErMultiplePriKey = 1068 - ErTooManyKeys = 1069 - ErTooManyKeyParts = 1070 - ErTooLongKey = 1071 - ErKeyColumnDoesNotExits = 1072 - ErBlobUsedAsKey = 1073 - ErTooBigFieldlength = 1074 - ErWrongAutoKey = 1075 - ErReady = 1076 - ErNormalShutdown = 1077 - ErGotSignal = 1078 - ErShutdownComplete = 1079 - ErForcingClose = 1080 - ErIpsockError = 1081 - ErNoSuchIndex = 1082 - ErWrongFieldTerminators = 1083 - ErBlobsAndNoTerminated = 1084 - ErTextfileNotReadable = 1085 - ErFileExistsError = 1086 - ErLoadInfo = 1087 - ErAlterInfo = 1088 - ErWrongSubKey = 1089 - ErCantRemoveAllFields = 1090 - ErCantDropFieldOrKey = 1091 - ErInsertInfo = 1092 - ErUpdateTableUsed = 1093 - ErNoSuchThread = 1094 - ErKillDeniedError = 1095 - ErNoTablesUsed = 1096 - ErTooBigSet = 1097 - ErNoUniqueLogfile = 1098 - ErTableNotLockedForWrite = 1099 - ErTableNotLocked = 1100 - ErBlobCantHaveDefault = 1101 - ErWrongDbName = 1102 - ErWrongTableName = 1103 - ErTooBigSelect = 1104 - ErUnknownError = 1105 - ErUnknownProcedure = 1106 - ErWrongParamcountToProcedure = 1107 - ErWrongParametersToProcedure = 1108 - ErUnknownTable = 1109 - ErFieldSpecifiedTwice = 1110 - ErInvalidGroupFuncUse = 1111 - ErUnsupportedExtension = 1112 - ErTableMustHaveColumns = 1113 - ErRecordFileFull = 1114 - ErUnknownCharacterSet = 1115 - ErTooManyTables = 1116 - ErTooManyFields = 1117 - ErTooBigRowsize = 1118 - ErStackOverrun = 1119 - ErWrongOuterJoin = 1120 - ErNullColumnInIndex = 1121 - ErCantFindUdf = 1122 - ErCantInitializeUdf = 1123 - ErUdfNoPaths = 1124 - ErUdfExists = 1125 - ErCantOpenLibrary = 1126 - ErCantFindDlEntry = 1127 - ErFunctionNotDefined = 1128 - ErHostIsBlocked = 1129 - ErHostNotPrivileged = 1130 - ErPasswordAnonymousUser = 1131 - ErPasswordNotAllowed = 1132 - ErPasswordNoMatch = 1133 - ErUpdateInfo = 1134 - ErCantCreateThread = 1135 - ErWrongValueCountOnRow = 1136 - ErCantReopenTable = 1137 - ErInvalidUseOfNull = 1138 - ErRegexpError = 1139 - ErMixOfGroupFuncAndFields = 1140 - ErNonexistingGrant = 1141 - ErTableaccessDeniedError = 1142 - ErColumnaccessDeniedError = 1143 - ErIllegalGrantForTable = 1144 - ErGrantWrongHostOrUser = 1145 - ErNoSuchTable = 1146 - ErNonexistingTableGrant = 1147 - ErNotAllowedCommand = 1148 - ErSyntaxError = 1149 - ErDelayedCantChangeLock = 1150 - ErTooManyDelayedThreads = 1151 - ErAbortingConnection = 1152 - ErNetPacketTooLarge = 1153 - ErNetReadErrorFromPipe = 1154 - ErNetFcntlError = 1155 - ErNetPacketsOutOfOrder = 1156 - ErNetUncompressError = 1157 - ErNetReadError = 1158 - ErNetReadInterrupted = 1159 - ErNetErrorOnWrite = 1160 - ErNetWriteInterrupted = 1161 - ErTooLongString = 1162 - ErTableCantHandleBlob = 1163 - ErTableCantHandleAutoIncrement = 1164 - ErDelayedInsertTableLocked = 1165 - ErWrongColumnName = 1166 - ErWrongKeyColumn = 1167 - ErWrongMrgTable = 1168 - ErDupUnique = 1169 - ErBlobKeyWithoutLength = 1170 - ErPrimaryCantHaveNull = 1171 - ErTooManyRows = 1172 - ErRequiresPrimaryKey = 1173 - ErNoRaidCompiled = 1174 - ErUpdateWithoutKeyInSafeMode = 1175 - ErKeyDoesNotExits = 1176 - ErCheckNoSuchTable = 1177 - ErCheckNotImplemented = 1178 - ErCantDoThisDuringAnTransaction = 1179 - ErErrorDuringCommit = 1180 - ErErrorDuringRollback = 1181 - ErErrorDuringFlushLogs = 1182 - ErErrorDuringCheckpoint = 1183 - ErNewAbortingConnection = 1184 - ErDumpNotImplemented = 1185 - ErFlushMasterBinlogClosed = 1186 - ErIndexRebuild = 1187 - ErMaster = 1188 - ErMasterNetRead = 1189 - ErMasterNetWrite = 1190 - ErFtMatchingKeyNotFound = 1191 - ErLockOrActiveTransaction = 1192 - ErUnknownSystemVariable = 1193 - ErCrashedOnUsage = 1194 - ErCrashedOnRepair = 1195 - ErWarningNotCompleteRollback = 1196 - ErTransCacheFull = 1197 - ErSlaveMustStop = 1198 - ErSlaveNotRunning = 1199 - ErBadSlave = 1200 - ErMasterInfo = 1201 - ErSlaveThread = 1202 - ErTooManyUserConnections = 1203 - ErSetConstantsOnly = 1204 - ErLockWaitTimeout = 1205 - ErLockTableFull = 1206 - ErReadOnlyTransaction = 1207 - ErDropDbWithReadLock = 1208 - ErCreateDbWithReadLock = 1209 - ErWrongArguments = 1210 - ErNoPermissionToCreateUser = 1211 - ErUnionTablesInDifferentDir = 1212 - ErLockDeadlock = 1213 - ErTableCantHandleFt = 1214 - ErCannotAddForeign = 1215 - ErNoReferencedRow = 1216 - ErRowIsReferenced = 1217 - ErConnectToMaster = 1218 - ErQueryOnMaster = 1219 - ErErrorWhenExecutingCommand = 1220 - ErWrongUsage = 1221 - ErWrongNumberOfColumnsInSelect = 1222 - ErCantUpdateWithReadlock = 1223 - ErMixingNotAllowed = 1224 - ErDupArgument = 1225 - ErUserLimitReached = 1226 - ErSpecificAccessDeniedError = 1227 - ErLocalVariable = 1228 - ErGlobalVariable = 1229 - ErNoDefault = 1230 - ErWrongValueForVar = 1231 - ErWrongTypeForVar = 1232 - ErVarCantBeRead = 1233 - ErCantUseOptionHere = 1234 - ErNotSupportedYet = 1235 - ErMasterFatalErrorReadingBinlog = 1236 - ErSlaveIgnoredTable = 1237 - ErIncorrectGlobalLocalVar = 1238 - ErWrongFkDef = 1239 - ErKeyRefDoNotMatchTableRef = 1240 - ErOperandColumns = 1241 - ErSubqueryNo1Row = 1242 - ErUnknownStmtHandler = 1243 - ErCorruptHelpDb = 1244 - ErCyclicReference = 1245 - ErAutoConvert = 1246 - ErIllegalReference = 1247 - ErDerivedMustHaveAlias = 1248 - ErSelectReduced = 1249 - ErTablenameNotAllowedHere = 1250 - ErNotSupportedAuthMode = 1251 - ErSpatialCantHaveNull = 1252 - ErCollationCharsetMismatch = 1253 - ErSlaveWasRunning = 1254 - ErSlaveWasNotRunning = 1255 - ErTooBigForUncompress = 1256 - ErZlibZMemError = 1257 - ErZlibZBufError = 1258 - ErZlibZDataError = 1259 - ErCutValueGroupConcat = 1260 - ErWarnTooFewRecords = 1261 - ErWarnTooManyRecords = 1262 - ErWarnNullToNotnull = 1263 - ErWarnDataOutOfRange = 1264 - WarnDataTruncated = 1265 - ErWarnUsingOtherHandler = 1266 - ErCantAggregate2collations = 1267 - ErDropUser = 1268 - ErRevokeGrants = 1269 - ErCantAggregate3collations = 1270 - ErCantAggregateNcollations = 1271 - ErVariableIsNotStruct = 1272 - ErUnknownCollation = 1273 - ErSlaveIgnoredSslParams = 1274 - ErServerIsInSecureAuthMode = 1275 - ErWarnFieldResolved = 1276 - ErBadSlaveUntilCond = 1277 - ErMissingSkipSlave = 1278 - ErUntilCondIgnored = 1279 - ErWrongNameForIndex = 1280 - ErWrongNameForCatalog = 1281 - ErWarnQcResize = 1282 - ErBadFtColumn = 1283 - ErUnknownKeyCache = 1284 - ErWarnHostnameWontWork = 1285 - ErUnknownStorageEngine = 1286 - ErWarnDeprecatedSyntax = 1287 - ErNonUpdatableTable = 1288 - ErFeatureDisabled = 1289 - ErOptionPreventsStatement = 1290 - ErDuplicatedValueInType = 1291 - ErTruncatedWrongValue = 1292 - ErTooMuchAutoTimestampCols = 1293 - ErInvalidOnUpdate = 1294 - ErUnsupportedPs = 1295 - ErGetErrmsg = 1296 - ErGetTemporaryErrmsg = 1297 - ErUnknownTimeZone = 1298 - ErWarnInvalidTimestamp = 1299 - ErInvalidCharacterString = 1300 - ErWarnAllowedPacketOverflowed = 1301 - ErConflictingDeclarations = 1302 - ErSpNoRecursiveCreate = 1303 - ErSpAlreadyExists = 1304 - ErSpDoesNotExist = 1305 - ErSpDropFailed = 1306 - ErSpStoreFailed = 1307 - ErSpLilabelMismatch = 1308 - ErSpLabelRedefine = 1309 - ErSpLabelMismatch = 1310 - ErSpUninitVar = 1311 - ErSpBadselect = 1312 - ErSpBadreturn = 1313 - ErSpBadstatement = 1314 - ErUpdateLogDeprecatedIgnored = 1315 - ErUpdateLogDeprecatedTranslated = 1316 - ErQueryInterrupted = 1317 - ErSpWrongNoOfArgs = 1318 - ErSpCondMismatch = 1319 - ErSpNoreturn = 1320 - ErSpNoreturnend = 1321 - ErSpBadCursorQuery = 1322 - ErSpBadCursorSelect = 1323 - ErSpCursorMismatch = 1324 - ErSpCursorAlreadyOpen = 1325 - ErSpCursorNotOpen = 1326 - ErSpUndeclaredVar = 1327 - ErSpWrongNoOfFetchArgs = 1328 - ErSpFetchNoData = 1329 - ErSpDupParam = 1330 - ErSpDupVar = 1331 - ErSpDupCond = 1332 - ErSpDupCurs = 1333 - ErSpCantAlter = 1334 - ErSpSubselectNyi = 1335 - ErStmtNotAllowedInSfOrTrg = 1336 - ErSpVarcondAfterCurshndlr = 1337 - ErSpCursorAfterHandler = 1338 - ErSpCaseNotFound = 1339 - ErFparserTooBigFile = 1340 - ErFparserBadHeader = 1341 - ErFparserEOFInComment = 1342 - ErFparserErrorInParameter = 1343 - ErFparserEOFInUnknownParameter = 1344 - ErViewNoExplain = 1345 - ErFrmUnknownType = 1346 - ErWrongObject = 1347 - ErNonupdateableColumn = 1348 - ErViewSelectDerived = 1349 - ErViewSelectClause = 1350 - ErViewSelectVariable = 1351 - ErViewSelectTmptable = 1352 - ErViewWrongList = 1353 - ErWarnViewMerge = 1354 - ErWarnViewWithoutKey = 1355 - ErViewInvalid = 1356 - ErSpNoDropSp = 1357 - ErSpGotoInHndlr = 1358 - ErTrgAlreadyExists = 1359 - ErTrgDoesNotExist = 1360 - ErTrgOnViewOrTempTable = 1361 - ErTrgCantChangeRow = 1362 - ErTrgNoSuchRowInTrg = 1363 - ErNoDefaultForField = 1364 - ErDivisionByZero = 1365 - ErTruncatedWrongValueForField = 1366 - ErIllegalValueForType = 1367 - ErViewNonupdCheck = 1368 - ErViewCheckFailed = 1369 - ErProcaccessDeniedError = 1370 - ErRelayLogFail = 1371 - ErPasswdLength = 1372 - ErUnknownTargetBinlog = 1373 - ErIoErrLogIndexRead = 1374 - ErBinlogPurgeProhibited = 1375 - ErFseekFail = 1376 - ErBinlogPurgeFatalErr = 1377 - ErLogInUse = 1378 - ErLogPurgeUnknownErr = 1379 - ErRelayLogInit = 1380 - ErNoBinaryLogging = 1381 - ErReservedSyntax = 1382 - ErWsasFailed = 1383 - ErDiffGroupsProc = 1384 - ErNoGroupForProc = 1385 - ErOrderWithProc = 1386 - ErLoggingProhibitChangingOf = 1387 - ErNoFileMapping = 1388 - ErWrongMagic = 1389 - ErPsManyParam = 1390 - ErKeyPart0 = 1391 - ErViewChecksum = 1392 - ErViewMultiupdate = 1393 - ErViewNoInsertFieldList = 1394 - ErViewDeleteMergeView = 1395 - ErCannotUser = 1396 - ErXaerNota = 1397 - ErXaerInval = 1398 - ErXaerRmfail = 1399 - ErXaerOutside = 1400 - ErXaerRmerr = 1401 - ErXaRbrollback = 1402 - ErNonexistingProcGrant = 1403 - ErProcAutoGrantFail = 1404 - ErProcAutoRevokeFail = 1405 - ErDataTooLong = 1406 - ErSpBadSQLstate = 1407 - ErStartup = 1408 - ErLoadFromFixedSizeRowsToVar = 1409 - ErCantCreateUserWithGrant = 1410 - ErWrongValueForType = 1411 - ErTableDefChanged = 1412 - ErSpDupHandler = 1413 - ErSpNotVarArg = 1414 - ErSpNoRetset = 1415 - ErCantCreateGeometryObject = 1416 - ErFailedRoutineBreakBinlog = 1417 - ErBinlogUnsafeRoutine = 1418 - ErBinlogCreateRoutineNeedSuper = 1419 - ErExecStmtWithOpenCursor = 1420 - ErStmtHasNoOpenCursor = 1421 - ErCommitNotAllowedInSfOrTrg = 1422 - ErNoDefaultForViewField = 1423 - ErSpNoRecursion = 1424 - ErTooBigScale = 1425 - ErTooBigPrecision = 1426 - ErMBiggerThanD = 1427 - ErWrongLockOfSystemTable = 1428 - ErConnectToForeignDataSource = 1429 - ErQueryOnForeignDataSource = 1430 - ErForeignDataSourceDoesntExist = 1431 - ErForeignDataStringInvalidCantCreate = 1432 - ErForeignDataStringInvalid = 1433 - ErCantCreateFederatedTable = 1434 - ErTrgInWrongSchema = 1435 - ErStackOverrunNeedMore = 1436 - ErTooLongBody = 1437 - ErWarnCantDropDefaultKeycache = 1438 - ErTooBigDisplaywidth = 1439 - ErXaerDupid = 1440 - ErDatetimeFunctionOverflow = 1441 - ErCantUpdateUsedTableInSfOrTrg = 1442 - ErViewPreventUpdate = 1443 - ErPsNoRecursion = 1444 - ErSpCantSetAutocommit = 1445 - ErMalformedDefiner = 1446 - ErViewFrmNoUser = 1447 - ErViewOtherUser = 1448 - ErNoSuchUser = 1449 - ErForbidSchemaChange = 1450 - ErRowIsReferenced2 = 1451 - ErNoReferencedRow2 = 1452 - ErSpBadVarShadow = 1453 - ErTrgNoDefiner = 1454 - ErOldFileFormat = 1455 - ErSpRecursionLimit = 1456 - ErSpProcTableCorrupt = 1457 - ErSpWrongName = 1458 - ErTableNeedsUpgrade = 1459 - ErSpNoAggregate = 1460 - ErMaxPreparedStmtCountReached = 1461 - ErViewRecursive = 1462 - ErNonGroupingFieldUsed = 1463 - ErTableCantHandleSpkeys = 1464 - ErNoTriggersOnSystemSchema = 1465 - ErRemovedSpaces = 1466 - ErAutoincReadFailed = 1467 - ErUsername = 1468 - ErHostname = 1469 - ErWrongStringLength = 1470 - ErNonInsertableTable = 1471 - ErAdminWrongMrgTable = 1472 - ErTooHighLevelOfNestingForSelect = 1473 - ErNameBecomesEmpty = 1474 - ErAmbiguousFieldTerm = 1475 - ErForeignServerExists = 1476 - ErForeignServerDoesntExist = 1477 - ErIllegalHaCreateOption = 1478 - ErPartitionRequiresValuesError = 1479 - ErPartitionWrongValuesError = 1480 - ErPartitionMaxvalueError = 1481 - ErPartitionSubpartitionError = 1482 - ErPartitionSubpartMixError = 1483 - ErPartitionWrongNoPartError = 1484 - ErPartitionWrongNoSubpartError = 1485 - ErWrongExprInPartitionFuncError = 1486 - ErNoConstExprInRangeOrListError = 1487 - ErFieldNotFoundPartError = 1488 - ErListOfFieldsOnlyInHashError = 1489 - ErInconsistentPartitionInfoError = 1490 - ErPartitionFuncNotAllowedError = 1491 - ErPartitionsMustBeDefinedError = 1492 - ErRangeNotIncreasingError = 1493 - ErInconsistentTypeOfFunctionsError = 1494 - ErMultipleDefConstInListPartError = 1495 - ErPartitionEntryError = 1496 - ErMixHandlerError = 1497 - ErPartitionNotDefinedError = 1498 - ErTooManyPartitionsError = 1499 - ErSubpartitionError = 1500 - ErCantCreateHandlerFile = 1501 - ErBlobFieldInPartFuncError = 1502 - ErUniqueKeyNeedAllFieldsInPf = 1503 - ErNoPartsError = 1504 - ErPartitionMgmtOnNonpartitioned = 1505 - ErForeignKeyOnPartitioned = 1506 - ErDropPartitionNonExistent = 1507 - ErDropLastPartition = 1508 - ErCoalesceOnlyOnHashPartition = 1509 - ErReorgHashOnlyOnSameNo = 1510 - ErReorgNoParamError = 1511 - ErOnlyOnRangeListPartition = 1512 - ErAddPartitionSubpartError = 1513 - ErAddPartitionNoNewPartition = 1514 - ErCoalescePartitionNoPartition = 1515 - ErReorgPartitionNotExist = 1516 - ErSameNamePartition = 1517 - ErNoBinlogError = 1518 - ErConsecutiveReorgPartitions = 1519 - ErReorgOutsideRange = 1520 - ErPartitionFunctionFailure = 1521 - ErPartStateError = 1522 - ErLimitedPartRange = 1523 - ErPluginIsNotLoaded = 1524 - ErWrongValue = 1525 - ErNoPartitionForGivenValue = 1526 - ErFilegroupOptionOnlyOnce = 1527 - ErCreateFilegroupFailed = 1528 - ErDropFilegroupFailed = 1529 - ErTablespaceAutoExtendError = 1530 - ErWrongSizeNumber = 1531 - ErSizeOverflowError = 1532 - ErAlterFilegroupFailed = 1533 - ErBinlogRowLoggingFailed = 1534 - ErBinlogRowWrongTableDef = 1535 - ErBinlogRowRbrToSbr = 1536 - ErEventAlreadyExists = 1537 - ErEventStoreFailed = 1538 - ErEventDoesNotExist = 1539 - ErEventCantAlter = 1540 - ErEventDropFailed = 1541 - ErEventIntervalNotPositiveOrTooBig = 1542 - ErEventEndsBeforeStarts = 1543 - ErEventExecTimeInThePast = 1544 - ErEventOpenTableFailed = 1545 - ErEventNeitherMExprNorMAt = 1546 - ErObsoleteColCountDoesntMatchCorrupted = 1547 - ErObsoleteCannotLoadFromTable = 1548 - ErEventCannotDelete = 1549 - ErEventCompileError = 1550 - ErEventSameName = 1551 - ErEventDataTooLong = 1552 - ErDropIndexFk = 1553 - ErWarnDeprecatedSyntaxWithVer = 1554 - ErCantWriteLockLogTable = 1555 - ErCantLockLogTable = 1556 - ErForeignDuplicateKeyOldUnused = 1557 - ErColCountDoesntMatchPleaseUpdate = 1558 - ErTempTablePreventsSwitchOutOfRbr = 1559 - ErStoredFunctionPreventsSwitchBinlogFormat = 1560 - ErNdbCantSwitchBinlogFormat = 1561 - ErPartitionNoTemporary = 1562 - ErPartitionConstDomainError = 1563 - ErPartitionFunctionIsNotAllowed = 1564 - ErDdlLogError = 1565 - ErNullInValuesLessThan = 1566 - ErWrongPartitionName = 1567 - ErCantChangeTxCharacteristics = 1568 - ErDupEntryAutoincrementCase = 1569 - ErEventModifyQueueError = 1570 - ErEventSetVarError = 1571 - ErPartitionMergeError = 1572 - ErCantActivateLog = 1573 - ErRbrNotAvailable = 1574 - ErBase64DecodeError = 1575 - ErEventRecursionForbidden = 1576 - ErEventsDbError = 1577 - ErOnlyIntegersAllowed = 1578 - ErUnsuportedLogEngine = 1579 - ErBadLogStatement = 1580 - ErCantRenameLogTable = 1581 - ErWrongParamcountToNativeFct = 1582 - ErWrongParametersToNativeFct = 1583 - ErWrongParametersToStoredFct = 1584 - ErNativeFctNameCollision = 1585 - ErDupEntryWithKeyName = 1586 - ErBinlogPurgeEmfile = 1587 - ErEventCannotCreateInThePast = 1588 - ErEventCannotAlterInThePast = 1589 - ErSlaveIncident = 1590 - ErNoPartitionForGivenValueSilent = 1591 - ErBinlogUnsafeStatement = 1592 - ErSlaveFatalError = 1593 - ErSlaveRelayLogReadFailure = 1594 - ErSlaveRelayLogWriteFailure = 1595 - ErSlaveCreateEventFailure = 1596 - ErSlaveMasterComFailure = 1597 - ErBinlogLoggingImpossible = 1598 - ErViewNoCreationCtx = 1599 - ErViewInvalidCreationCtx = 1600 - ErSrInvalidCreationCtx = 1601 - ErTrgCorruptedFile = 1602 - ErTrgNoCreationCtx = 1603 - ErTrgInvalidCreationCtx = 1604 - ErEventInvalidCreationCtx = 1605 - ErTrgCantOpenTable = 1606 - ErCantCreateSroutine = 1607 - ErNeverUsed = 1608 - ErNoFormatDescriptionEventBeforeBinlogStatement = 1609 - ErSlaveCorruptEvent = 1610 - ErLoadDataInvalidColumn = 1611 - ErLogPurgeNoFile = 1612 - ErXaRbtimeout = 1613 - ErXaRbdeadlock = 1614 - ErNeedReprepare = 1615 - ErDelayedNotSupported = 1616 - WarnNoMasterInfo = 1617 - WarnOptionIgnored = 1618 - WarnPluginDeleteBuiltin = 1619 - WarnPluginBusy = 1620 - ErVariableIsReadonly = 1621 - ErWarnEngineTransactionRollback = 1622 - ErSlaveHeartbeatFailure = 1623 - ErSlaveHeartbeatValueOutOfRange = 1624 - ErNdbReplicationSchemaError = 1625 - ErConflictFnParseError = 1626 - ErExceptionsWriteError = 1627 - ErTooLongTableComment = 1628 - ErTooLongFieldComment = 1629 - ErFuncInexistentNameCollision = 1630 - ErDatabaseName = 1631 - ErTableName = 1632 - ErPartitionName = 1633 - ErSubpartitionName = 1634 - ErTemporaryName = 1635 - ErRenamedName = 1636 - ErTooManyConcurrentTrxs = 1637 - WarnNonASCIISeparatorNotImplemented = 1638 - ErDebugSyncTimeout = 1639 - ErDebugSyncHitLimit = 1640 - ErDupSignalSet = 1641 - ErSignalWarn = 1642 - ErSignalNotFound = 1643 - ErSignalException = 1644 - ErResignalWithoutActiveHandler = 1645 - ErSignalBadConditionType = 1646 - WarnCondItemTruncated = 1647 - ErCondItemTooLong = 1648 - ErUnknownLocale = 1649 - ErSlaveIgnoreServerIds = 1650 - ErQueryCacheDisabled = 1651 - ErSameNamePartitionField = 1652 - ErPartitionColumnListError = 1653 - ErWrongTypeColumnValueError = 1654 - ErTooManyPartitionFuncFieldsError = 1655 - ErMaxvalueInValuesIn = 1656 - ErTooManyValuesError = 1657 - ErRowSinglePartitionFieldError = 1658 - ErFieldTypeNotAllowedAsPartitionField = 1659 - ErPartitionFieldsTooLong = 1660 - ErBinlogRowEngineAndStmtEngine = 1661 - ErBinlogRowModeAndStmtEngine = 1662 - ErBinlogUnsafeAndStmtEngine = 1663 - ErBinlogRowInjectionAndStmtEngine = 1664 - ErBinlogStmtModeAndRowEngine = 1665 - ErBinlogRowInjectionAndStmtMode = 1666 - ErBinlogMultipleEnginesAndSelfLoggingEngine = 1667 - ErBinlogUnsafeLimit = 1668 - ErBinlogUnsafeInsertDelayed = 1669 - ErBinlogUnsafeSystemTable = 1670 - ErBinlogUnsafeAutoincColumns = 1671 - ErBinlogUnsafeUdf = 1672 - ErBinlogUnsafeSystemVariable = 1673 - ErBinlogUnsafeSystemFunction = 1674 - ErBinlogUnsafeNontransAfterTrans = 1675 - ErMessageAndStatement = 1676 - ErSlaveConversionFailed = 1677 - ErSlaveCantCreateConversion = 1678 - ErInsideTransactionPreventsSwitchBinlogFormat = 1679 - ErPathLength = 1680 - ErWarnDeprecatedSyntaxNoReplacement = 1681 - ErWrongNativeTableStructure = 1682 - ErWrongPerfschemaUsage = 1683 - ErWarnISSkippedTable = 1684 - ErInsideTransactionPreventsSwitchBinlogDirect = 1685 - ErStoredFunctionPreventsSwitchBinlogDirect = 1686 - ErSpatialMustHaveGeomCol = 1687 - ErTooLongIndexComment = 1688 - ErLockAborted = 1689 - ErDataOutOfRange = 1690 - ErWrongSpvarTypeInLimit = 1691 - ErBinlogUnsafeMultipleEnginesAndSelfLoggingEngine = 1692 - ErBinlogUnsafeMixedStatement = 1693 - ErInsideTransactionPreventsSwitchSQLLogBin = 1694 - ErStoredFunctionPreventsSwitchSQLLogBin = 1695 - ErFailedReadFromParFile = 1696 - ErValuesIsNotIntTypeError = 1697 - ErAccessDeniedNoPasswordError = 1698 - ErSetPasswordAuthPlugin = 1699 - ErGrantPluginUserExists = 1700 - ErTruncateIllegalFk = 1701 - ErPluginIsPermanent = 1702 - ErSlaveHeartbeatValueOutOfRangeMin = 1703 - ErSlaveHeartbeatValueOutOfRangeMax = 1704 - ErStmtCacheFull = 1705 - ErMultiUpdateKeyConflict = 1706 - ErTableNeedsRebuild = 1707 - WarnOptionBelowLimit = 1708 - ErIndexColumnTooLong = 1709 - ErErrorInTriggerBody = 1710 - ErErrorInUnknownTriggerBody = 1711 - ErIndexCorrupt = 1712 - ErUndoRecordTooBig = 1713 - ErBinlogUnsafeInsertIgnoreSelect = 1714 - ErBinlogUnsafeInsertSelectUpdate = 1715 - ErBinlogUnsafeReplaceSelect = 1716 - ErBinlogUnsafeCreateIgnoreSelect = 1717 - ErBinlogUnsafeCreateReplaceSelect = 1718 - ErBinlogUnsafeUpdateIgnore = 1719 - ErPluginNoUninstall = 1720 - ErPluginNoInstall = 1721 - ErBinlogUnsafeWriteAutoincSelect = 1722 - ErBinlogUnsafeCreateSelectAutoinc = 1723 - ErBinlogUnsafeInsertTwoKeys = 1724 - ErTableInFkCheck = 1725 - ErUnsupportedEngine = 1726 - ErBinlogUnsafeAutoincNotFirst = 1727 - ErCannotLoadFromTableV2 = 1728 - ErMasterDelayValueOutOfRange = 1729 - ErOnlyFdAndRbrEventsAllowedInBinlogStatement = 1730 - ErPartitionExchangeDifferentOption = 1731 - ErPartitionExchangePartTable = 1732 - ErPartitionExchangeTempTable = 1733 - ErPartitionInsteadOfSubpartition = 1734 - ErUnknownPartition = 1735 - ErTablesDifferentMetadata = 1736 - ErRowDoesNotMatchPartition = 1737 - ErBinlogCacheSizeGreaterThanMax = 1738 - ErWarnIndexNotApplicable = 1739 - ErPartitionExchangeForeignKey = 1740 - ErNoSuchKeyValue = 1741 - ErRplInfoDataTooLong = 1742 - ErNetworkReadEventChecksumFailure = 1743 - ErBinlogReadEventChecksumFailure = 1744 - ErBinlogStmtCacheSizeGreaterThanMax = 1745 - ErCantUpdateTableInCreateTableSelect = 1746 - ErPartitionClauseOnNonpartitioned = 1747 - ErRowDoesNotMatchGivenPartitionSet = 1748 - ErNoSuchPartitionunused = 1749 - ErChangeRplInfoRepositoryFailure = 1750 - ErWarningNotCompleteRollbackWithCreatedTempTable = 1751 - ErWarningNotCompleteRollbackWithDroppedTempTable = 1752 - ErMtsFeatureIsNotSupported = 1753 - ErMtsUpdatedDbsGreaterMax = 1754 - ErMtsCantParallel = 1755 - ErMtsInconsistentData = 1756 - ErFulltextNotSupportedWithPartitioning = 1757 - ErDaInvalidConditionNumber = 1758 - ErInsecurePlainText = 1759 - ErInsecureChangeMaster = 1760 - ErForeignDuplicateKeyWithChildInfo = 1761 - ErForeignDuplicateKeyWithoutChildInfo = 1762 - ErSQLthreadWithSecureSlave = 1763 - ErTableHasNoFt = 1764 - ErVariableNotSettableInSfOrTrigger = 1765 - ErVariableNotSettableInTransaction = 1766 - ErGtidNextIsNotInGtidNextList = 1767 - ErCantChangeGtidNextInTransactionWhenGtidNextListIsNull = 1768 - ErSetStatementCannotInvokeFunction = 1769 - ErGtidNextCantBeAutomaticIfGtidNextListIsNonNull = 1770 - ErSkippingLoggedTransaction = 1771 - ErMalformedGtidSetSpecification = 1772 - ErMalformedGtidSetEncoding = 1773 - ErMalformedGtidSpecification = 1774 - ErGnoExhausted = 1775 - ErBadSlaveAutoPosition = 1776 - ErAutoPositionRequiresGtidModeOn = 1777 - ErCantDoImplicitCommitInTrxWhenGtidNextIsSet = 1778 - ErGtidMode2Or3RequiresEnforceGtidConsistencyOn = 1779 - ErGtidModeRequiresBinlog = 1780 - ErCantSetGtidNextToGtidWhenGtidModeIsOff = 1781 - ErCantSetGtidNextToAnonymousWhenGtidModeIsOn = 1782 - ErCantSetGtidNextListToNonNullWhenGtidModeIsOff = 1783 - ErFoundGtidEventWhenGtidModeIsOff = 1784 - ErGtidUnsafeNonTransactionalTable = 1785 - ErGtidUnsafeCreateSelect = 1786 - ErGtidUnsafeCreateDropTemporaryTableInTransaction = 1787 - ErGtidModeCanOnlyChangeOneStepAtATime = 1788 - ErMasterHasPurgedRequiredGtids = 1789 - ErCantSetGtidNextWhenOwningGtid = 1790 - ErUnknownExplainFormat = 1791 - ErCantExecuteInReadOnlyTransaction = 1792 - ErTooLongTablePartitionComment = 1793 - ErSlaveConfiguration = 1794 - ErInnodbFtLimit = 1795 - ErInnodbNoFtTempTable = 1796 - ErInnodbFtWrongDocidColumn = 1797 - ErInnodbFtWrongDocidIndex = 1798 - ErInnodbOnlineLogTooBig = 1799 - ErUnknownAlterAlgorithm = 1800 - ErUnknownAlterLock = 1801 - ErMtsChangeMasterCantRunWithGaps = 1802 - ErMtsRecoveryFailure = 1803 - ErMtsResetWorkers = 1804 - ErColCountDoesntMatchCorruptedV2 = 1805 - ErSlaveSilentRetryTransaction = 1806 - ErDiscardFkChecksRunning = 1807 - ErTableSchemaMismatch = 1808 - ErTableInSystemTablespace = 1809 - ErIoReadError = 1810 - ErIoWriteError = 1811 - ErTablespaceMissing = 1812 - ErTablespaceExists = 1813 - ErTablespaceDiscarded = 1814 - ErInternalError = 1815 - ErInnodbImportError = 1816 - ErInnodbIndexCorrupt = 1817 - ErInvalidYearColumnLength = 1818 - ErNotValidPassword = 1819 - ErMustChangePassword = 1820 - ErFkNoIndexChild = 1821 - ErFkNoIndexParent = 1822 - ErFkFailAddSystem = 1823 - ErFkCannotOpenParent = 1824 - ErFkIncorrectOption = 1825 - ErFkDupName = 1826 - ErPasswordFormat = 1827 - ErFkColumnCannotDrop = 1828 - ErFkColumnCannotDropChild = 1829 - ErFkColumnNotNull = 1830 - ErDupIndex = 1831 - ErFkColumnCannotChange = 1832 - ErFkColumnCannotChangeChild = 1833 - ErFkCannotDeleteParent = 1834 - ErMalformedPacket = 1835 - ErReadOnlyMode = 1836 - ErGtidNextTypeUndefinedGroup = 1837 - ErVariableNotSettableInSp = 1838 - ErCantSetGtidPurgedWhenGtidModeIsOff = 1839 - ErCantSetGtidPurgedWhenGtidExecutedIsNotEmpty = 1840 - ErCantSetGtidPurgedWhenOwnedGtidsIsNotEmpty = 1841 - ErGtidPurgedWasChanged = 1842 - ErGtidExecutedWasChanged = 1843 - ErBinlogStmtModeAndNoReplTables = 1844 - ErAlterOperationNotSupported = 1845 - ErAlterOperationNotSupportedReason = 1846 - ErAlterOperationNotSupportedReasonCopy = 1847 - ErAlterOperationNotSupportedReasonPartition = 1848 - ErAlterOperationNotSupportedReasonFkRename = 1849 - ErAlterOperationNotSupportedReasonColumnType = 1850 - ErAlterOperationNotSupportedReasonFkCheck = 1851 - ErAlterOperationNotSupportedReasonIgnore = 1852 - ErAlterOperationNotSupportedReasonNopk = 1853 - ErAlterOperationNotSupportedReasonAutoinc = 1854 - ErAlterOperationNotSupportedReasonHiddenFts = 1855 - ErAlterOperationNotSupportedReasonChangeFts = 1856 - ErAlterOperationNotSupportedReasonFts = 1857 - ErSQLSlaveSkipCounterNotSettableInGtidMode = 1858 - ErDupUnknownInIndex = 1859 - ErIdentCausesTooLongPath = 1860 - ErAlterOperationNotSupportedReasonNotNull = 1861 - ErMustChangePasswordLogin = 1862 - ErRowInWrongPartition = 1863 - ErErrorLast = 1863 + ErrErrorFirst uint16 = 1000 + ErrHashchk = 1000 + ErrNisamchk = 1001 + ErrNo = 1002 + ErrYes = 1003 + ErrCantCreateFile = 1004 + ErrCantCreateTable = 1005 + ErrCantCreateDb = 1006 + ErrDbCreateExists = 1007 + ErrDbDropExists = 1008 + ErrDbDropDelete = 1009 + ErrDbDropRmdir = 1010 + ErrCantDeleteFile = 1011 + ErrCantFindSystemRec = 1012 + ErrCantGetStat = 1013 + ErrCantGetWd = 1014 + ErrCantLock = 1015 + ErrCantOpenFile = 1016 + ErrFileNotFound = 1017 + ErrCantReadDir = 1018 + ErrCantSetWd = 1019 + ErrCheckread = 1020 + ErrDiskFull = 1021 + ErrDupKey = 1022 + ErrErrorOnClose = 1023 + ErrErrorOnRead = 1024 + ErrErrorOnRename = 1025 + ErrErrorOnWrite = 1026 + ErrFileUsed = 1027 + ErrFilsortAbort = 1028 + ErrFormNotFound = 1029 + ErrGetErrno = 1030 + ErrIllegalHa = 1031 + ErrKeyNotFound = 1032 + ErrNotFormFile = 1033 + ErrNotKeyfile = 1034 + ErrOldKeyfile = 1035 + ErrOpenAsReadonly = 1036 + ErrOutofmemory = 1037 + ErrOutOfSortmemory = 1038 + ErrUnexpectedEOF = 1039 + ErrConCount = 1040 + ErrOutOfResources = 1041 + ErrBadHost = 1042 + ErrHandshake = 1043 + ErrDbaccessDenied = 1044 + ErrAccessDenied = 1045 + ErrNoDb = 1046 + ErrUnknownCom = 1047 + ErrBadNull = 1048 + ErrBadDb = 1049 + ErrTableExists = 1050 + ErrBadTable = 1051 + ErrNonUniq = 1052 + ErrServerShutdown = 1053 + ErrBadField = 1054 + ErrWrongFieldWithGroup = 1055 + ErrWrongGroupField = 1056 + ErrWrongSumSelect = 1057 + ErrWrongValueCount = 1058 + ErrTooLongIdent = 1059 + ErrDupFieldname = 1060 + ErrDupKeyname = 1061 + ErrDupEntry = 1062 + ErrWrongFieldSpec = 1063 + ErrParse = 1064 + ErrEmptyQuery = 1065 + ErrNonuniqTable = 1066 + ErrInvalidDefault = 1067 + ErrMultiplePriKey = 1068 + ErrTooManyKeys = 1069 + ErrTooManyKeyParts = 1070 + ErrTooLongKey = 1071 + ErrKeyColumnDoesNotExits = 1072 + ErrBlobUsedAsKey = 1073 + ErrTooBigFieldlength = 1074 + ErrWrongAutoKey = 1075 + ErrReady = 1076 + ErrNormalShutdown = 1077 + ErrGotSignal = 1078 + ErrShutdownComplete = 1079 + ErrForcingClose = 1080 + ErrIpsock = 1081 + ErrNoSuchIndex = 1082 + ErrWrongFieldTerminators = 1083 + ErrBlobsAndNoTerminated = 1084 + ErrTextfileNotReadable = 1085 + ErrFileExists = 1086 + ErrLoadInfo = 1087 + ErrAlterInfo = 1088 + ErrWrongSubKey = 1089 + ErrCantRemoveAllFields = 1090 + ErrCantDropFieldOrKey = 1091 + ErrInsertInfo = 1092 + ErrUpdateTableUsed = 1093 + ErrNoSuchThread = 1094 + ErrKillDenied = 1095 + ErrNoTablesUsed = 1096 + ErrTooBigSet = 1097 + ErrNoUniqueLogfile = 1098 + ErrTableNotLockedForWrite = 1099 + ErrTableNotLocked = 1100 + ErrBlobCantHaveDefault = 1101 + ErrWrongDbName = 1102 + ErrWrongTableName = 1103 + ErrTooBigSelect = 1104 + ErrUnknown = 1105 + ErrUnknownProcedure = 1106 + ErrWrongParamcountToProcedure = 1107 + ErrWrongParametersToProcedure = 1108 + ErrUnknownTable = 1109 + ErrFieldSpecifiedTwice = 1110 + ErrInvalidGroupFuncUse = 1111 + ErrUnsupportedExtension = 1112 + ErrTableMustHaveColumns = 1113 + ErrRecordFileFull = 1114 + ErrUnknownCharacterSet = 1115 + ErrTooManyTables = 1116 + ErrTooManyFields = 1117 + ErrTooBigRowsize = 1118 + ErrStackOverrun = 1119 + ErrWrongOuterJoin = 1120 + ErrNullColumnInIndex = 1121 + ErrCantFindUdf = 1122 + ErrCantInitializeUdf = 1123 + ErrUdfNoPaths = 1124 + ErrUdfExists = 1125 + ErrCantOpenLibrary = 1126 + ErrCantFindDlEntry = 1127 + ErrFunctionNotDefined = 1128 + ErrHostIsBlocked = 1129 + ErrHostNotPrivileged = 1130 + ErrPasswordAnonymousUser = 1131 + ErrPasswordNotAllowed = 1132 + ErrPasswordNoMatch = 1133 + ErrUpdateInfo = 1134 + ErrCantCreateThread = 1135 + ErrWrongValueCountOnRow = 1136 + ErrCantReopenTable = 1137 + ErrInvalidUseOfNull = 1138 + ErrRegexp = 1139 + ErrMixOfGroupFuncAndFields = 1140 + ErrNonexistingGrant = 1141 + ErrTableaccessDenied = 1142 + ErrColumnaccessDenied = 1143 + ErrIllegalGrantForTable = 1144 + ErrGrantWrongHostOrUser = 1145 + ErrNoSuchTable = 1146 + ErrNonexistingTableGrant = 1147 + ErrNotAllowedCommand = 1148 + ErrSyntax = 1149 + ErrDelayedCantChangeLock = 1150 + ErrTooManyDelayedThreads = 1151 + ErrAbortingConnection = 1152 + ErrNetPacketTooLarge = 1153 + ErrNetReadErrorFromPipe = 1154 + ErrNetFcntl = 1155 + ErrNetPacketsOutOfOrder = 1156 + ErrNetUncompress = 1157 + ErrNetRead = 1158 + ErrNetReadInterrupted = 1159 + ErrNetErrorOnWrite = 1160 + ErrNetWriteInterrupted = 1161 + ErrTooLongString = 1162 + ErrTableCantHandleBlob = 1163 + ErrTableCantHandleAutoIncrement = 1164 + ErrDelayedInsertTableLocked = 1165 + ErrWrongColumnName = 1166 + ErrWrongKeyColumn = 1167 + ErrWrongMrgTable = 1168 + ErrDupUnique = 1169 + ErrBlobKeyWithoutLength = 1170 + ErrPrimaryCantHaveNull = 1171 + ErrTooManyRows = 1172 + ErrRequiresPrimaryKey = 1173 + ErrNoRaidCompiled = 1174 + ErrUpdateWithoutKeyInSafeMode = 1175 + ErrKeyDoesNotExits = 1176 + ErrCheckNoSuchTable = 1177 + ErrCheckNotImplemented = 1178 + ErrCantDoThisDuringAnTransaction = 1179 + ErrErrorDuringCommit = 1180 + ErrErrorDuringRollback = 1181 + ErrErrorDuringFlushLogs = 1182 + ErrErrorDuringCheckpoint = 1183 + ErrNewAbortingConnection = 1184 + ErrDumpNotImplemented = 1185 + ErrFlushMasterBinlogClosed = 1186 + ErrIndexRebuild = 1187 + ErrMaster = 1188 + ErrMasterNetRead = 1189 + ErrMasterNetWrite = 1190 + ErrFtMatchingKeyNotFound = 1191 + ErrLockOrActiveTransaction = 1192 + ErrUnknownSystemVariable = 1193 + ErrCrashedOnUsage = 1194 + ErrCrashedOnRepair = 1195 + ErrWarningNotCompleteRollback = 1196 + ErrTransCacheFull = 1197 + ErrSlaveMustStop = 1198 + ErrSlaveNotRunning = 1199 + ErrBadSlave = 1200 + ErrMasterInfo = 1201 + ErrSlaveThread = 1202 + ErrTooManyUserConnections = 1203 + ErrSetConstantsOnly = 1204 + ErrLockWaitTimeout = 1205 + ErrLockTableFull = 1206 + ErrReadOnlyTransaction = 1207 + ErrDropDbWithReadLock = 1208 + ErrCreateDbWithReadLock = 1209 + ErrWrongArguments = 1210 + ErrNoPermissionToCreateUser = 1211 + ErrUnionTablesInDifferentDir = 1212 + ErrLockDeadlock = 1213 + ErrTableCantHandleFt = 1214 + ErrCannotAddForeign = 1215 + ErrNoReferencedRow = 1216 + ErrRowIsReferenced = 1217 + ErrConnectToMaster = 1218 + ErrQueryOnMaster = 1219 + ErrErrorWhenExecutingCommand = 1220 + ErrWrongUsage = 1221 + ErrWrongNumberOfColumnsInSelect = 1222 + ErrCantUpdateWithReadlock = 1223 + ErrMixingNotAllowed = 1224 + ErrDupArgument = 1225 + ErrUserLimitReached = 1226 + ErrSpecificAccessDenied = 1227 + ErrLocalVariable = 1228 + ErrGlobalVariable = 1229 + ErrNoDefault = 1230 + ErrWrongValueForVar = 1231 + ErrWrongTypeForVar = 1232 + ErrVarCantBeRead = 1233 + ErrCantUseOptionHere = 1234 + ErrNotSupportedYet = 1235 + ErrMasterFatalErrorReadingBinlog = 1236 + ErrSlaveIgnoredTable = 1237 + ErrIncorrectGlobalLocalVar = 1238 + ErrWrongFkDef = 1239 + ErrKeyRefDoNotMatchTableRef = 1240 + ErrOperandColumns = 1241 + ErrSubqueryNo1Row = 1242 + ErrUnknownStmtHandler = 1243 + ErrCorruptHelpDb = 1244 + ErrCyclicReference = 1245 + ErrAutoConvert = 1246 + ErrIllegalReference = 1247 + ErrDerivedMustHaveAlias = 1248 + ErrSelectReduced = 1249 + ErrTablenameNotAllowedHere = 1250 + ErrNotSupportedAuthMode = 1251 + ErrSpatialCantHaveNull = 1252 + ErrCollationCharsetMismatch = 1253 + ErrSlaveWasRunning = 1254 + ErrSlaveWasNotRunning = 1255 + ErrTooBigForUncompress = 1256 + ErrZlibZMem = 1257 + ErrZlibZBuf = 1258 + ErrZlibZData = 1259 + ErrCutValueGroupConcat = 1260 + ErrWarnTooFewRecords = 1261 + ErrWarnTooManyRecords = 1262 + ErrWarnNullToNotnull = 1263 + ErrWarnDataOutOfRange = 1264 + WarnDataTruncated = 1265 + ErrWarnUsingOtherHandler = 1266 + ErrCantAggregate2collations = 1267 + ErrDropUser = 1268 + ErrRevokeGrants = 1269 + ErrCantAggregate3collations = 1270 + ErrCantAggregateNcollations = 1271 + ErrVariableIsNotStruct = 1272 + ErrUnknownCollation = 1273 + ErrSlaveIgnoredSslParams = 1274 + ErrServerIsInSecureAuthMode = 1275 + ErrWarnFieldResolved = 1276 + ErrBadSlaveUntilCond = 1277 + ErrMissingSkipSlave = 1278 + ErrUntilCondIgnored = 1279 + ErrWrongNameForIndex = 1280 + ErrWrongNameForCatalog = 1281 + ErrWarnQcResize = 1282 + ErrBadFtColumn = 1283 + ErrUnknownKeyCache = 1284 + ErrWarnHostnameWontWork = 1285 + ErrUnknownStorageEngine = 1286 + ErrWarnDeprecatedSyntax = 1287 + ErrNonUpdatableTable = 1288 + ErrFeatureDisabled = 1289 + ErrOptionPreventsStatement = 1290 + ErrDuplicatedValueInType = 1291 + ErrTruncatedWrongValue = 1292 + ErrTooMuchAutoTimestampCols = 1293 + ErrInvalidOnUpdate = 1294 + ErrUnsupportedPs = 1295 + ErrGetErrmsg = 1296 + ErrGetTemporaryErrmsg = 1297 + ErrUnknownTimeZone = 1298 + ErrWarnInvalidTimestamp = 1299 + ErrInvalidCharacterString = 1300 + ErrWarnAllowedPacketOverflowed = 1301 + ErrConflictingDeclarations = 1302 + ErrSpNoRecursiveCreate = 1303 + ErrSpAlreadyExists = 1304 + ErrSpDoesNotExist = 1305 + ErrSpDropFailed = 1306 + ErrSpStoreFailed = 1307 + ErrSpLilabelMismatch = 1308 + ErrSpLabelRedefine = 1309 + ErrSpLabelMismatch = 1310 + ErrSpUninitVar = 1311 + ErrSpBadselect = 1312 + ErrSpBadreturn = 1313 + ErrSpBadstatement = 1314 + ErrUpdateLogDeprecatedIgnored = 1315 + ErrUpdateLogDeprecatedTranslated = 1316 + ErrQueryInterrupted = 1317 + ErrSpWrongNoOfArgs = 1318 + ErrSpCondMismatch = 1319 + ErrSpNoreturn = 1320 + ErrSpNoreturnend = 1321 + ErrSpBadCursorQuery = 1322 + ErrSpBadCursorSelect = 1323 + ErrSpCursorMismatch = 1324 + ErrSpCursorAlreadyOpen = 1325 + ErrSpCursorNotOpen = 1326 + ErrSpUndeclaredVar = 1327 + ErrSpWrongNoOfFetchArgs = 1328 + ErrSpFetchNoData = 1329 + ErrSpDupParam = 1330 + ErrSpDupVar = 1331 + ErrSpDupCond = 1332 + ErrSpDupCurs = 1333 + ErrSpCantAlter = 1334 + ErrSpSubselectNyi = 1335 + ErrStmtNotAllowedInSfOrTrg = 1336 + ErrSpVarcondAfterCurshndlr = 1337 + ErrSpCursorAfterHandler = 1338 + ErrSpCaseNotFound = 1339 + ErrFparserTooBigFile = 1340 + ErrFparserBadHeader = 1341 + ErrFparserEOFInComment = 1342 + ErrFparserErrorInParameter = 1343 + ErrFparserEOFInUnknownParameter = 1344 + ErrViewNoExplain = 1345 + ErrFrmUnknownType = 1346 + ErrWrongObject = 1347 + ErrNonupdateableColumn = 1348 + ErrViewSelectDerived = 1349 + ErrViewSelectClause = 1350 + ErrViewSelectVariable = 1351 + ErrViewSelectTmptable = 1352 + ErrViewWrongList = 1353 + ErrWarnViewMerge = 1354 + ErrWarnViewWithoutKey = 1355 + ErrViewInvalid = 1356 + ErrSpNoDropSp = 1357 + ErrSpGotoInHndlr = 1358 + ErrTrgAlreadyExists = 1359 + ErrTrgDoesNotExist = 1360 + ErrTrgOnViewOrTempTable = 1361 + ErrTrgCantChangeRow = 1362 + ErrTrgNoSuchRowInTrg = 1363 + ErrNoDefaultForField = 1364 + ErrDivisionByZero = 1365 + ErrTruncatedWrongValueForField = 1366 + ErrIllegalValueForType = 1367 + ErrViewNonupdCheck = 1368 + ErrViewCheckFailed = 1369 + ErrProcaccessDenied = 1370 + ErrRelayLogFail = 1371 + ErrPasswdLength = 1372 + ErrUnknownTargetBinlog = 1373 + ErrIoErrLogIndexRead = 1374 + ErrBinlogPurgeProhibited = 1375 + ErrFseekFail = 1376 + ErrBinlogPurgeFatalErr = 1377 + ErrLogInUse = 1378 + ErrLogPurgeUnknownErr = 1379 + ErrRelayLogInit = 1380 + ErrNoBinaryLogging = 1381 + ErrReservedSyntax = 1382 + ErrWsasFailed = 1383 + ErrDiffGroupsProc = 1384 + ErrNoGroupForProc = 1385 + ErrOrderWithProc = 1386 + ErrLoggingProhibitChangingOf = 1387 + ErrNoFileMapping = 1388 + ErrWrongMagic = 1389 + ErrPsManyParam = 1390 + ErrKeyPart0 = 1391 + ErrViewChecksum = 1392 + ErrViewMultiupdate = 1393 + ErrViewNoInsertFieldList = 1394 + ErrViewDeleteMergeView = 1395 + ErrCannotUser = 1396 + ErrXaerNota = 1397 + ErrXaerInval = 1398 + ErrXaerRmfail = 1399 + ErrXaerOutside = 1400 + ErrXaerRmerr = 1401 + ErrXaRbrollback = 1402 + ErrNonexistingProcGrant = 1403 + ErrProcAutoGrantFail = 1404 + ErrProcAutoRevokeFail = 1405 + ErrDataTooLong = 1406 + ErrSpBadSQLstate = 1407 + ErrStartup = 1408 + ErrLoadFromFixedSizeRowsToVar = 1409 + ErrCantCreateUserWithGrant = 1410 + ErrWrongValueForType = 1411 + ErrTableDefChanged = 1412 + ErrSpDupHandler = 1413 + ErrSpNotVarArg = 1414 + ErrSpNoRetset = 1415 + ErrCantCreateGeometryObject = 1416 + ErrFailedRoutineBreakBinlog = 1417 + ErrBinlogUnsafeRoutine = 1418 + ErrBinlogCreateRoutineNeedSuper = 1419 + ErrExecStmtWithOpenCursor = 1420 + ErrStmtHasNoOpenCursor = 1421 + ErrCommitNotAllowedInSfOrTrg = 1422 + ErrNoDefaultForViewField = 1423 + ErrSpNoRecursion = 1424 + ErrTooBigScale = 1425 + ErrTooBigPrecision = 1426 + ErrMBiggerThanD = 1427 + ErrWrongLockOfSystemTable = 1428 + ErrConnectToForeignDataSource = 1429 + ErrQueryOnForeignDataSource = 1430 + ErrForeignDataSourceDoesntExist = 1431 + ErrForeignDataStringInvalidCantCreate = 1432 + ErrForeignDataStringInvalid = 1433 + ErrCantCreateFederatedTable = 1434 + ErrTrgInWrongSchema = 1435 + ErrStackOverrunNeedMore = 1436 + ErrTooLongBody = 1437 + ErrWarnCantDropDefaultKeycache = 1438 + ErrTooBigDisplaywidth = 1439 + ErrXaerDupid = 1440 + ErrDatetimeFunctionOverflow = 1441 + ErrCantUpdateUsedTableInSfOrTrg = 1442 + ErrViewPreventUpdate = 1443 + ErrPsNoRecursion = 1444 + ErrSpCantSetAutocommit = 1445 + ErrMalformedDefiner = 1446 + ErrViewFrmNoUser = 1447 + ErrViewOtherUser = 1448 + ErrNoSuchUser = 1449 + ErrForbidSchemaChange = 1450 + ErrRowIsReferenced2 = 1451 + ErrNoReferencedRow2 = 1452 + ErrSpBadVarShadow = 1453 + ErrTrgNoDefiner = 1454 + ErrOldFileFormat = 1455 + ErrSpRecursionLimit = 1456 + ErrSpProcTableCorrupt = 1457 + ErrSpWrongName = 1458 + ErrTableNeedsUpgrade = 1459 + ErrSpNoAggregate = 1460 + ErrMaxPreparedStmtCountReached = 1461 + ErrViewRecursive = 1462 + ErrNonGroupingFieldUsed = 1463 + ErrTableCantHandleSpkeys = 1464 + ErrNoTriggersOnSystemSchema = 1465 + ErrRemovedSpaces = 1466 + ErrAutoincReadFailed = 1467 + ErrUsername = 1468 + ErrHostname = 1469 + ErrWrongStringLength = 1470 + ErrNonInsertableTable = 1471 + ErrAdminWrongMrgTable = 1472 + ErrTooHighLevelOfNestingForSelect = 1473 + ErrNameBecomesEmpty = 1474 + ErrAmbiguousFieldTerm = 1475 + ErrForeignServerExists = 1476 + ErrForeignServerDoesntExist = 1477 + ErrIllegalHaCreateOption = 1478 + ErrPartitionRequiresValues = 1479 + ErrPartitionWrongValues = 1480 + ErrPartitionMaxvalue = 1481 + ErrPartitionSubpartition = 1482 + ErrPartitionSubpartMix = 1483 + ErrPartitionWrongNoPart = 1484 + ErrPartitionWrongNoSubpart = 1485 + ErrWrongExprInPartitionFunc = 1486 + ErrNoConstExprInRangeOrList = 1487 + ErrFieldNotFoundPart = 1488 + ErrListOfFieldsOnlyInHash = 1489 + ErrInconsistentPartitionInfo = 1490 + ErrPartitionFuncNotAllowed = 1491 + ErrPartitionsMustBeDefined = 1492 + ErrRangeNotIncreasing = 1493 + ErrInconsistentTypeOfFunctions = 1494 + ErrMultipleDefConstInListPart = 1495 + ErrPartitionEntry = 1496 + ErrMixHandler = 1497 + ErrPartitionNotDefined = 1498 + ErrTooManyPartitions = 1499 + ErrSubpartition = 1500 + ErrCantCreateHandlerFile = 1501 + ErrBlobFieldInPartFunc = 1502 + ErrUniqueKeyNeedAllFieldsInPf = 1503 + ErrNoParts = 1504 + ErrPartitionMgmtOnNonpartitioned = 1505 + ErrForeignKeyOnPartitioned = 1506 + ErrDropPartitionNonExistent = 1507 + ErrDropLastPartition = 1508 + ErrCoalesceOnlyOnHashPartition = 1509 + ErrReorgHashOnlyOnSameNo = 1510 + ErrReorgNoParam = 1511 + ErrOnlyOnRangeListPartition = 1512 + ErrAddPartitionSubpart = 1513 + ErrAddPartitionNoNewPartition = 1514 + ErrCoalescePartitionNoPartition = 1515 + ErrReorgPartitionNotExist = 1516 + ErrSameNamePartition = 1517 + ErrNoBinlog = 1518 + ErrConsecutiveReorgPartitions = 1519 + ErrReorgOutsideRange = 1520 + ErrPartitionFunctionFailure = 1521 + ErrPartState = 1522 + ErrLimitedPartRange = 1523 + ErrPluginIsNotLoaded = 1524 + ErrWrongValue = 1525 + ErrNoPartitionForGivenValue = 1526 + ErrFilegroupOptionOnlyOnce = 1527 + ErrCreateFilegroupFailed = 1528 + ErrDropFilegroupFailed = 1529 + ErrTablespaceAutoExtend = 1530 + ErrWrongSizeNumber = 1531 + ErrSizeOverflow = 1532 + ErrAlterFilegroupFailed = 1533 + ErrBinlogRowLoggingFailed = 1534 + ErrBinlogRowWrongTableDef = 1535 + ErrBinlogRowRbrToSbr = 1536 + ErrEventAlreadyExists = 1537 + ErrEventStoreFailed = 1538 + ErrEventDoesNotExist = 1539 + ErrEventCantAlter = 1540 + ErrEventDropFailed = 1541 + ErrEventIntervalNotPositiveOrTooBig = 1542 + ErrEventEndsBeforeStarts = 1543 + ErrEventExecTimeInThePast = 1544 + ErrEventOpenTableFailed = 1545 + ErrEventNeitherMExprNorMAt = 1546 + ErrObsoleteColCountDoesntMatchCorrupted = 1547 + ErrObsoleteCannotLoadFromTable = 1548 + ErrEventCannotDelete = 1549 + ErrEventCompile = 1550 + ErrEventSameName = 1551 + ErrEventDataTooLong = 1552 + ErrDropIndexFk = 1553 + ErrWarnDeprecatedSyntaxWithVer = 1554 + ErrCantWriteLockLogTable = 1555 + ErrCantLockLogTable = 1556 + ErrForeignDuplicateKeyOldUnused = 1557 + ErrColCountDoesntMatchPleaseUpdate = 1558 + ErrTempTablePreventsSwitchOutOfRbr = 1559 + ErrStoredFunctionPreventsSwitchBinlogFormat = 1560 + ErrNdbCantSwitchBinlogFormat = 1561 + ErrPartitionNoTemporary = 1562 + ErrPartitionConstDomain = 1563 + ErrPartitionFunctionIsNotAllowed = 1564 + ErrDdlLog = 1565 + ErrNullInValuesLessThan = 1566 + ErrWrongPartitionName = 1567 + ErrCantChangeTxCharacteristics = 1568 + ErrDupEntryAutoincrementCase = 1569 + ErrEventModifyQueue = 1570 + ErrEventSetVar = 1571 + ErrPartitionMerge = 1572 + ErrCantActivateLog = 1573 + ErrRbrNotAvailable = 1574 + ErrBase64Decode = 1575 + ErrEventRecursionForbidden = 1576 + ErrEventsDb = 1577 + ErrOnlyIntegersAllowed = 1578 + ErrUnsuportedLogEngine = 1579 + ErrBadLogStatement = 1580 + ErrCantRenameLogTable = 1581 + ErrWrongParamcountToNativeFct = 1582 + ErrWrongParametersToNativeFct = 1583 + ErrWrongParametersToStoredFct = 1584 + ErrNativeFctNameCollision = 1585 + ErrDupEntryWithKeyName = 1586 + ErrBinlogPurgeEmfile = 1587 + ErrEventCannotCreateInThePast = 1588 + ErrEventCannotAlterInThePast = 1589 + ErrSlaveIncident = 1590 + ErrNoPartitionForGivenValueSilent = 1591 + ErrBinlogUnsafeStatement = 1592 + ErrSlaveFatal = 1593 + ErrSlaveRelayLogReadFailure = 1594 + ErrSlaveRelayLogWriteFailure = 1595 + ErrSlaveCreateEventFailure = 1596 + ErrSlaveMasterComFailure = 1597 + ErrBinlogLoggingImpossible = 1598 + ErrViewNoCreationCtx = 1599 + ErrViewInvalidCreationCtx = 1600 + ErrSrInvalidCreationCtx = 1601 + ErrTrgCorruptedFile = 1602 + ErrTrgNoCreationCtx = 1603 + ErrTrgInvalidCreationCtx = 1604 + ErrEventInvalidCreationCtx = 1605 + ErrTrgCantOpenTable = 1606 + ErrCantCreateSroutine = 1607 + ErrNeverUsed = 1608 + ErrNoFormatDescriptionEventBeforeBinlogStatement = 1609 + ErrSlaveCorruptEvent = 1610 + ErrLoadDataInvalidColumn = 1611 + ErrLogPurgeNoFile = 1612 + ErrXaRbtimeout = 1613 + ErrXaRbdeadlock = 1614 + ErrNeedReprepare = 1615 + ErrDelayedNotSupported = 1616 + WarnNoMasterInfo = 1617 + WarnOptionIgnored = 1618 + WarnPluginDeleteBuiltin = 1619 + WarnPluginBusy = 1620 + ErrVariableIsReadonly = 1621 + ErrWarnEngineTransactionRollback = 1622 + ErrSlaveHeartbeatFailure = 1623 + ErrSlaveHeartbeatValueOutOfRange = 1624 + ErrNdbReplicationSchema = 1625 + ErrConflictFnParse = 1626 + ErrExceptionsWrite = 1627 + ErrTooLongTableComment = 1628 + ErrTooLongFieldComment = 1629 + ErrFuncInexistentNameCollision = 1630 + ErrDatabaseName = 1631 + ErrTableName = 1632 + ErrPartitionName = 1633 + ErrSubpartitionName = 1634 + ErrTemporaryName = 1635 + ErrRenamedName = 1636 + ErrTooManyConcurrentTrxs = 1637 + WarnNonASCIISeparatorNotImplemented = 1638 + ErrDebugSyncTimeout = 1639 + ErrDebugSyncHitLimit = 1640 + ErrDupSignalSet = 1641 + ErrSignalWarn = 1642 + ErrSignalNotFound = 1643 + ErrSignalException = 1644 + ErrResignalWithoutActiveHandler = 1645 + ErrSignalBadConditionType = 1646 + WarnCondItemTruncated = 1647 + ErrCondItemTooLong = 1648 + ErrUnknownLocale = 1649 + ErrSlaveIgnoreServerIds = 1650 + ErrQueryCacheDisabled = 1651 + ErrSameNamePartitionField = 1652 + ErrPartitionColumnList = 1653 + ErrWrongTypeColumnValue = 1654 + ErrTooManyPartitionFuncFields = 1655 + ErrMaxvalueInValuesIn = 1656 + ErrTooManyValues = 1657 + ErrRowSinglePartitionField = 1658 + ErrFieldTypeNotAllowedAsPartitionField = 1659 + ErrPartitionFieldsTooLong = 1660 + ErrBinlogRowEngineAndStmtEngine = 1661 + ErrBinlogRowModeAndStmtEngine = 1662 + ErrBinlogUnsafeAndStmtEngine = 1663 + ErrBinlogRowInjectionAndStmtEngine = 1664 + ErrBinlogStmtModeAndRowEngine = 1665 + ErrBinlogRowInjectionAndStmtMode = 1666 + ErrBinlogMultipleEnginesAndSelfLoggingEngine = 1667 + ErrBinlogUnsafeLimit = 1668 + ErrBinlogUnsafeInsertDelayed = 1669 + ErrBinlogUnsafeSystemTable = 1670 + ErrBinlogUnsafeAutoincColumns = 1671 + ErrBinlogUnsafeUdf = 1672 + ErrBinlogUnsafeSystemVariable = 1673 + ErrBinlogUnsafeSystemFunction = 1674 + ErrBinlogUnsafeNontransAfterTrans = 1675 + ErrMessageAndStatement = 1676 + ErrSlaveConversionFailed = 1677 + ErrSlaveCantCreateConversion = 1678 + ErrInsideTransactionPreventsSwitchBinlogFormat = 1679 + ErrPathLength = 1680 + ErrWarnDeprecatedSyntaxNoReplacement = 1681 + ErrWrongNativeTableStructure = 1682 + ErrWrongPerfschemaUsage = 1683 + ErrWarnISSkippedTable = 1684 + ErrInsideTransactionPreventsSwitchBinlogDirect = 1685 + ErrStoredFunctionPreventsSwitchBinlogDirect = 1686 + ErrSpatialMustHaveGeomCol = 1687 + ErrTooLongIndexComment = 1688 + ErrLockAborted = 1689 + ErrDataOutOfRange = 1690 + ErrWrongSpvarTypeInLimit = 1691 + ErrBinlogUnsafeMultipleEnginesAndSelfLoggingEngine = 1692 + ErrBinlogUnsafeMixedStatement = 1693 + ErrInsideTransactionPreventsSwitchSQLLogBin = 1694 + ErrStoredFunctionPreventsSwitchSQLLogBin = 1695 + ErrFailedReadFromParFile = 1696 + ErrValuesIsNotIntType = 1697 + ErrAccessDeniedNoPassword = 1698 + ErrSetPasswordAuthPlugin = 1699 + ErrGrantPluginUserExists = 1700 + ErrTruncateIllegalFk = 1701 + ErrPluginIsPermanent = 1702 + ErrSlaveHeartbeatValueOutOfRangeMin = 1703 + ErrSlaveHeartbeatValueOutOfRangeMax = 1704 + ErrStmtCacheFull = 1705 + ErrMultiUpdateKeyConflict = 1706 + ErrTableNeedsRebuild = 1707 + WarnOptionBelowLimit = 1708 + ErrIndexColumnTooLong = 1709 + ErrErrorInTriggerBody = 1710 + ErrErrorInUnknownTriggerBody = 1711 + ErrIndexCorrupt = 1712 + ErrUndoRecordTooBig = 1713 + ErrBinlogUnsafeInsertIgnoreSelect = 1714 + ErrBinlogUnsafeInsertSelectUpdate = 1715 + ErrBinlogUnsafeReplaceSelect = 1716 + ErrBinlogUnsafeCreateIgnoreSelect = 1717 + ErrBinlogUnsafeCreateReplaceSelect = 1718 + ErrBinlogUnsafeUpdateIgnore = 1719 + ErrPluginNoUninstall = 1720 + ErrPluginNoInstall = 1721 + ErrBinlogUnsafeWriteAutoincSelect = 1722 + ErrBinlogUnsafeCreateSelectAutoinc = 1723 + ErrBinlogUnsafeInsertTwoKeys = 1724 + ErrTableInFkCheck = 1725 + ErrUnsupportedEngine = 1726 + ErrBinlogUnsafeAutoincNotFirst = 1727 + ErrCannotLoadFromTableV2 = 1728 + ErrMasterDelayValueOutOfRange = 1729 + ErrOnlyFdAndRbrEventsAllowedInBinlogStatement = 1730 + ErrPartitionExchangeDifferentOption = 1731 + ErrPartitionExchangePartTable = 1732 + ErrPartitionExchangeTempTable = 1733 + ErrPartitionInsteadOfSubpartition = 1734 + ErrUnknownPartition = 1735 + ErrTablesDifferentMetadata = 1736 + ErrRowDoesNotMatchPartition = 1737 + ErrBinlogCacheSizeGreaterThanMax = 1738 + ErrWarnIndexNotApplicable = 1739 + ErrPartitionExchangeForeignKey = 1740 + ErrNoSuchKeyValue = 1741 + ErrRplInfoDataTooLong = 1742 + ErrNetworkReadEventChecksumFailure = 1743 + ErrBinlogReadEventChecksumFailure = 1744 + ErrBinlogStmtCacheSizeGreaterThanMax = 1745 + ErrCantUpdateTableInCreateTableSelect = 1746 + ErrPartitionClauseOnNonpartitioned = 1747 + ErrRowDoesNotMatchGivenPartitionSet = 1748 + ErrNoSuchPartitionunused = 1749 + ErrChangeRplInfoRepositoryFailure = 1750 + ErrWarningNotCompleteRollbackWithCreatedTempTable = 1751 + ErrWarningNotCompleteRollbackWithDroppedTempTable = 1752 + ErrMtsFeatureIsNotSupported = 1753 + ErrMtsUpdatedDbsGreaterMax = 1754 + ErrMtsCantParallel = 1755 + ErrMtsInconsistentData = 1756 + ErrFulltextNotSupportedWithPartitioning = 1757 + ErrDaInvalidConditionNumber = 1758 + ErrInsecurePlainText = 1759 + ErrInsecureChangeMaster = 1760 + ErrForeignDuplicateKeyWithChildInfo = 1761 + ErrForeignDuplicateKeyWithoutChildInfo = 1762 + ErrSQLthreadWithSecureSlave = 1763 + ErrTableHasNoFt = 1764 + ErrVariableNotSettableInSfOrTrigger = 1765 + ErrVariableNotSettableInTransaction = 1766 + ErrGtidNextIsNotInGtidNextList = 1767 + ErrCantChangeGtidNextInTransactionWhenGtidNextListIsNull = 1768 + ErrSetStatementCannotInvokeFunction = 1769 + ErrGtidNextCantBeAutomaticIfGtidNextListIsNonNull = 1770 + ErrSkippingLoggedTransaction = 1771 + ErrMalformedGtidSetSpecification = 1772 + ErrMalformedGtidSetEncoding = 1773 + ErrMalformedGtidSpecification = 1774 + ErrGnoExhausted = 1775 + ErrBadSlaveAutoPosition = 1776 + ErrAutoPositionRequiresGtidModeOn = 1777 + ErrCantDoImplicitCommitInTrxWhenGtidNextIsSet = 1778 + ErrGtidMode2Or3RequiresEnforceGtidConsistencyOn = 1779 + ErrGtidModeRequiresBinlog = 1780 + ErrCantSetGtidNextToGtidWhenGtidModeIsOff = 1781 + ErrCantSetGtidNextToAnonymousWhenGtidModeIsOn = 1782 + ErrCantSetGtidNextListToNonNullWhenGtidModeIsOff = 1783 + ErrFoundGtidEventWhenGtidModeIsOff = 1784 + ErrGtidUnsafeNonTransactionalTable = 1785 + ErrGtidUnsafeCreateSelect = 1786 + ErrGtidUnsafeCreateDropTemporaryTableInTransaction = 1787 + ErrGtidModeCanOnlyChangeOneStepAtATime = 1788 + ErrMasterHasPurgedRequiredGtids = 1789 + ErrCantSetGtidNextWhenOwningGtid = 1790 + ErrUnknownExplainFormat = 1791 + ErrCantExecuteInReadOnlyTransaction = 1792 + ErrTooLongTablePartitionComment = 1793 + ErrSlaveConfiguration = 1794 + ErrInnodbFtLimit = 1795 + ErrInnodbNoFtTempTable = 1796 + ErrInnodbFtWrongDocidColumn = 1797 + ErrInnodbFtWrongDocidIndex = 1798 + ErrInnodbOnlineLogTooBig = 1799 + ErrUnknownAlterAlgorithm = 1800 + ErrUnknownAlterLock = 1801 + ErrMtsChangeMasterCantRunWithGaps = 1802 + ErrMtsRecoveryFailure = 1803 + ErrMtsResetWorkers = 1804 + ErrColCountDoesntMatchCorruptedV2 = 1805 + ErrSlaveSilentRetryTransaction = 1806 + ErrDiscardFkChecksRunning = 1807 + ErrTableSchemaMismatch = 1808 + ErrTableInSystemTablespace = 1809 + ErrIoRead = 1810 + ErrIoWrite = 1811 + ErrTablespaceMissing = 1812 + ErrTablespaceExists = 1813 + ErrTablespaceDiscarded = 1814 + ErrInternal = 1815 + ErrInnodbImport = 1816 + ErrInnodbIndexCorrupt = 1817 + ErrInvalidYearColumnLength = 1818 + ErrNotValidPassword = 1819 + ErrMustChangePassword = 1820 + ErrFkNoIndexChild = 1821 + ErrFkNoIndexParent = 1822 + ErrFkFailAddSystem = 1823 + ErrFkCannotOpenParent = 1824 + ErrFkIncorrectOption = 1825 + ErrFkDupName = 1826 + ErrPasswordFormat = 1827 + ErrFkColumnCannotDrop = 1828 + ErrFkColumnCannotDropChild = 1829 + ErrFkColumnNotNull = 1830 + ErrDupIndex = 1831 + ErrFkColumnCannotChange = 1832 + ErrFkColumnCannotChangeChild = 1833 + ErrFkCannotDeleteParent = 1834 + ErrMalformedPacket = 1835 + ErrReadOnlyMode = 1836 + ErrGtidNextTypeUndefinedGroup = 1837 + ErrVariableNotSettableInSp = 1838 + ErrCantSetGtidPurgedWhenGtidModeIsOff = 1839 + ErrCantSetGtidPurgedWhenGtidExecutedIsNotEmpty = 1840 + ErrCantSetGtidPurgedWhenOwnedGtidsIsNotEmpty = 1841 + ErrGtidPurgedWasChanged = 1842 + ErrGtidExecutedWasChanged = 1843 + ErrBinlogStmtModeAndNoReplTables = 1844 + ErrAlterOperationNotSupported = 1845 + ErrAlterOperationNotSupportedReason = 1846 + ErrAlterOperationNotSupportedReasonCopy = 1847 + ErrAlterOperationNotSupportedReasonPartition = 1848 + ErrAlterOperationNotSupportedReasonFkRename = 1849 + ErrAlterOperationNotSupportedReasonColumnType = 1850 + ErrAlterOperationNotSupportedReasonFkCheck = 1851 + ErrAlterOperationNotSupportedReasonIgnore = 1852 + ErrAlterOperationNotSupportedReasonNopk = 1853 + ErrAlterOperationNotSupportedReasonAutoinc = 1854 + ErrAlterOperationNotSupportedReasonHiddenFts = 1855 + ErrAlterOperationNotSupportedReasonChangeFts = 1856 + ErrAlterOperationNotSupportedReasonFts = 1857 + ErrSQLSlaveSkipCounterNotSettableInGtidMode = 1858 + ErrDupUnknownInIndex = 1859 + ErrIdentCausesTooLongPath = 1860 + ErrAlterOperationNotSupportedReasonNotNull = 1861 + ErrMustChangePasswordLogin = 1862 + ErrRowInWrongPartition = 1863 + ErrErrorLast = 1863 ) diff --git a/mysqldef/errname.go b/mysqldef/errname.go index 88936d2fa3..bc98fed16b 100644 --- a/mysqldef/errname.go +++ b/mysqldef/errname.go @@ -15,868 +15,868 @@ package mysqldef // MySQLErrName maps error code to MySQL error messages. var MySQLErrName = map[uint16]string{ - ErHashchk: "hashchk", - ErNisamchk: "isamchk", - ErNo: "NO", - ErYes: "YES", - ErCantCreateFile: "Can't create file '%-.200s' (errno: %d - %s)", - ErCantCreateTable: "Can't create table '%-.200s' (errno: %d)", - ErCantCreateDb: "Can't create database '%-.192s' (errno: %d)", - ErDbCreateExists: "Can't create database '%-.192s'; database exists", - ErDbDropExists: "Can't drop database '%-.192s'; database doesn't exist", - ErDbDropDelete: "Error dropping database (can't delete '%-.192s', errno: %d)", - ErDbDropRmdir: "Error dropping database (can't rmdir '%-.192s', errno: %d)", - ErCantDeleteFile: "Error on delete of '%-.192s' (errno: %d - %s)", - ErCantFindSystemRec: "Can't read record in system table", - ErCantGetStat: "Can't get status of '%-.200s' (errno: %d - %s)", - ErCantGetWd: "Can't get working directory (errno: %d - %s)", - ErCantLock: "Can't lock file (errno: %d - %s)", - ErCantOpenFile: "Can't open file: '%-.200s' (errno: %d - %s)", - ErFileNotFound: "Can't find file: '%-.200s' (errno: %d - %s)", - ErCantReadDir: "Can't read dir of '%-.192s' (errno: %d - %s)", - ErCantSetWd: "Can't change dir to '%-.192s' (errno: %d - %s)", - ErCheckread: "Record has changed since last read in table '%-.192s'", - ErDiskFull: "Disk full (%s); waiting for someone to free some space... (errno: %d - %s)", - ErDupKey: "Can't write; duplicate key in table '%-.192s'", - ErErrorOnClose: "Error on close of '%-.192s' (errno: %d - %s)", - ErErrorOnRead: "Error reading file '%-.200s' (errno: %d - %s)", - ErErrorOnRename: "Error on rename of '%-.210s' to '%-.210s' (errno: %d - %s)", - ErErrorOnWrite: "Error writing file '%-.200s' (errno: %d - %s)", - ErFileUsed: "'%-.192s' is locked against change", - ErFilsortAbort: "Sort aborted", - ErFormNotFound: "View '%-.192s' doesn't exist for '%-.192s'", - ErGetErrno: "Got error %d from storage engine", - ErIllegalHa: "Table storage engine for '%-.192s' doesn't have this option", - ErKeyNotFound: "Can't find record in '%-.192s'", - ErNotFormFile: "Incorrect information in file: '%-.200s'", - ErNotKeyfile: "Incorrect key file for table '%-.200s'; try to repair it", - ErOldKeyfile: "Old key file for table '%-.192s'; repair it!", - ErOpenAsReadonly: "Table '%-.192s' is read only", - ErOutofmemory: "Out of memory; restart server and try again (needed %d bytes)", - ErOutOfSortmemory: "Out of sort memory, consider increasing server sort buffer size", - ErUnexpectedEOF: "Unexpected EOF found when reading file '%-.192s' (errno: %d - %s)", - ErConCountError: "Too many connections", - ErOutOfResources: "Out of memory; check if mysqld or some other process uses all available memory; if not, you may have to use 'ulimit' to allow mysqld to use more memory or you can add more swap space", - ErBadHostError: "Can't get hostname for your address", - ErHandshakeError: "Bad handshake", - ErDbaccessDeniedError: "Access denied for user '%-.48s'@'%-.64s' to database '%-.192s'", - ErAccessDeniedError: "Access denied for user '%-.48s'@'%-.64s' (using password: %s)", - ErNoDbError: "No database selected", - ErUnknownComError: "Unknown command", - ErBadNullError: "Column '%-.192s' cannot be null", - ErBadDbError: "Unknown database '%-.192s'", - ErTableExistsError: "Table '%-.192s' already exists", - ErBadTableError: "Unknown table '%-.100s'", - ErNonUniqError: "Column '%-.192s' in %-.192s is ambiguous", - ErServerShutdown: "Server shutdown in progress", - ErBadFieldError: "Unknown column '%-.192s' in '%-.192s'", - ErWrongFieldWithGroup: "'%-.192s' isn't in GROUP BY", - ErWrongGroupField: "Can't group on '%-.192s'", - ErWrongSumSelect: "Statement has sum functions and columns in same statement", - ErWrongValueCount: "Column count doesn't match value count", - ErTooLongIdent: "Identifier name '%-.100s' is too long", - ErDupFieldname: "Duplicate column name '%-.192s'", - ErDupKeyname: "Duplicate key name '%-.192s'", - ErDupEntry: "Duplicate entry '%-.192s' for key %d", - ErWrongFieldSpec: "Incorrect column specifier for column '%-.192s'", - ErParseError: "%s near '%-.80s' at line %d", - ErEmptyQuery: "Query was empty", - ErNonuniqTable: "Not unique table/alias: '%-.192s'", - ErInvalidDefault: "Invalid default value for '%-.192s'", - ErMultiplePriKey: "Multiple primary key defined", - ErTooManyKeys: "Too many keys specified; max %d keys allowed", - ErTooManyKeyParts: "Too many key parts specified; max %d parts allowed", - ErTooLongKey: "Specified key was too long; max key length is %d bytes", - ErKeyColumnDoesNotExits: "Key column '%-.192s' doesn't exist in table", - ErBlobUsedAsKey: "BLOB column '%-.192s' can't be used in key specification with the used table type", - ErTooBigFieldlength: "Column length too big for column '%-.192s' (max = %lu); use BLOB or TEXT instead", - ErWrongAutoKey: "Incorrect table definition; there can be only one auto column and it must be defined as a key", - ErReady: "%s: ready for connections.\nVersion: '%s' socket: '%s' port: %d", - ErNormalShutdown: "%s: Normal shutdown\n", - ErGotSignal: "%s: Got signal %d. Aborting!\n", - ErShutdownComplete: "%s: Shutdown complete\n", - ErForcingClose: "%s: Forcing close of thread %ld user: '%-.48s'\n", - ErIpsockError: "Can't create IP socket", - ErNoSuchIndex: "Table '%-.192s' has no index like the one used in CREATE INDEX; recreate the table", - ErWrongFieldTerminators: "Field separator argument is not what is expected; check the manual", - ErBlobsAndNoTerminated: "You can't use fixed rowlength with BLOBs; please use 'fields terminated by'", - ErTextfileNotReadable: "The file '%-.128s' must be in the database directory or be readable by all", - ErFileExistsError: "File '%-.200s' already exists", - ErLoadInfo: "Records: %ld Deleted: %ld Skipped: %ld Warnings: %ld", - ErAlterInfo: "Records: %ld Duplicates: %ld", - ErWrongSubKey: "Incorrect prefix key; the used key part isn't a string, the used length is longer than the key part, or the storage engine doesn't support unique prefix keys", - ErCantRemoveAllFields: "You can't delete all columns with ALTER TABLE; use DROP TABLE instead", - ErCantDropFieldOrKey: "Can't DROP '%-.192s'; check that column/key exists", - ErInsertInfo: "Records: %ld Duplicates: %ld Warnings: %ld", - ErUpdateTableUsed: "You can't specify target table '%-.192s' for update in FROM clause", - ErNoSuchThread: "Unknown thread id: %lu", - ErKillDeniedError: "You are not owner of thread %lu", - ErNoTablesUsed: "No tables used", - ErTooBigSet: "Too many strings for column %-.192s and SET", - ErNoUniqueLogfile: "Can't generate a unique log-filename %-.200s.(1-999)\n", - ErTableNotLockedForWrite: "Table '%-.192s' was locked with a READ lock and can't be updated", - ErTableNotLocked: "Table '%-.192s' was not locked with LOCK TABLES", - ErBlobCantHaveDefault: "BLOB/TEXT column '%-.192s' can't have a default value", - ErWrongDbName: "Incorrect database name '%-.100s'", - ErWrongTableName: "Incorrect table name '%-.100s'", - ErTooBigSelect: "The SELECT would examine more than MAXJOINSIZE rows; check your WHERE and use SET SQLBIGSELECTS=1 or SET MAXJOINSIZE=# if the SELECT is okay", - ErUnknownError: "Unknown error", - ErUnknownProcedure: "Unknown procedure '%-.192s'", - ErWrongParamcountToProcedure: "Incorrect parameter count to procedure '%-.192s'", - ErWrongParametersToProcedure: "Incorrect parameters to procedure '%-.192s'", - ErUnknownTable: "Unknown table '%-.192s' in %-.32s", - ErFieldSpecifiedTwice: "Column '%-.192s' specified twice", - ErInvalidGroupFuncUse: "Invalid use of group function", - ErUnsupportedExtension: "Table '%-.192s' uses an extension that doesn't exist in this MySQL version", - ErTableMustHaveColumns: "A table must have at least 1 column", - ErRecordFileFull: "The table '%-.192s' is full", - ErUnknownCharacterSet: "Unknown character set: '%-.64s'", - ErTooManyTables: "Too many tables; MySQL can only use %d tables in a join", - ErTooManyFields: "Too many columns", - ErTooBigRowsize: "Row size too large. The maximum row size for the used table type, not counting BLOBs, is %ld. This includes storage overhead, check the manual. You have to change some columns to TEXT or BLOBs", - ErStackOverrun: "Thread stack overrun: Used: %ld of a %ld stack. Use 'mysqld --threadStack=#' to specify a bigger stack if needed", - ErWrongOuterJoin: "Cross dependency found in OUTER JOIN; examine your ON conditions", - ErNullColumnInIndex: "Table handler doesn't support NULL in given index. Please change column '%-.192s' to be NOT NULL or use another handler", - ErCantFindUdf: "Can't load function '%-.192s'", - ErCantInitializeUdf: "Can't initialize function '%-.192s'; %-.80s", - ErUdfNoPaths: "No paths allowed for shared library", - ErUdfExists: "Function '%-.192s' already exists", - ErCantOpenLibrary: "Can't open shared library '%-.192s' (errno: %d %-.128s)", - ErCantFindDlEntry: "Can't find symbol '%-.128s' in library", - ErFunctionNotDefined: "Function '%-.192s' is not defined", - ErHostIsBlocked: "Host '%-.64s' is blocked because of many connection errors; unblock with 'mysqladmin flush-hosts'", - ErHostNotPrivileged: "Host '%-.64s' is not allowed to connect to this MySQL server", - ErPasswordAnonymousUser: "You are using MySQL as an anonymous user and anonymous users are not allowed to change passwords", - ErPasswordNotAllowed: "You must have privileges to update tables in the mysql database to be able to change passwords for others", - ErPasswordNoMatch: "Can't find any matching row in the user table", - ErUpdateInfo: "Rows matched: %ld Changed: %ld Warnings: %ld", - ErCantCreateThread: "Can't create a new thread (errno %d); if you are not out of available memory, you can consult the manual for a possible OS-dependent bug", - ErWrongValueCountOnRow: "Column count doesn't match value count at row %ld", - ErCantReopenTable: "Can't reopen table: '%-.192s'", - ErInvalidUseOfNull: "Invalid use of NULL value", - ErRegexpError: "Got error '%-.64s' from regexp", - ErMixOfGroupFuncAndFields: "Mixing of GROUP columns (MIN(),MAX(),COUNT(),...) with no GROUP columns is illegal if there is no GROUP BY clause", - ErNonexistingGrant: "There is no such grant defined for user '%-.48s' on host '%-.64s'", - ErTableaccessDeniedError: "%-.128s command denied to user '%-.48s'@'%-.64s' for table '%-.64s'", - ErColumnaccessDeniedError: "%-.16s command denied to user '%-.48s'@'%-.64s' for column '%-.192s' in table '%-.192s'", - ErIllegalGrantForTable: "Illegal GRANT/REVOKE command; please consult the manual to see which privileges can be used", - ErGrantWrongHostOrUser: "The host or user argument to GRANT is too long", - ErNoSuchTable: "Table '%-.192s.%-.192s' doesn't exist", - ErNonexistingTableGrant: "There is no such grant defined for user '%-.48s' on host '%-.64s' on table '%-.192s'", - ErNotAllowedCommand: "The used command is not allowed with this MySQL version", - ErSyntaxError: "You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use", - ErDelayedCantChangeLock: "Delayed insert thread couldn't get requested lock for table %-.192s", - ErTooManyDelayedThreads: "Too many delayed threads in use", - ErAbortingConnection: "Aborted connection %ld to db: '%-.192s' user: '%-.48s' (%-.64s)", - ErNetPacketTooLarge: "Got a packet bigger than 'maxAllowedPacket' bytes", - ErNetReadErrorFromPipe: "Got a read error from the connection pipe", - ErNetFcntlError: "Got an error from fcntl()", - ErNetPacketsOutOfOrder: "Got packets out of order", - ErNetUncompressError: "Couldn't uncompress communication packet", - ErNetReadError: "Got an error reading communication packets", - ErNetReadInterrupted: "Got timeout reading communication packets", - ErNetErrorOnWrite: "Got an error writing communication packets", - ErNetWriteInterrupted: "Got timeout writing communication packets", - ErTooLongString: "Result string is longer than 'maxAllowedPacket' bytes", - ErTableCantHandleBlob: "The used table type doesn't support BLOB/TEXT columns", - ErTableCantHandleAutoIncrement: "The used table type doesn't support AUTOINCREMENT columns", - ErDelayedInsertTableLocked: "INSERT DELAYED can't be used with table '%-.192s' because it is locked with LOCK TABLES", - ErWrongColumnName: "Incorrect column name '%-.100s'", - ErWrongKeyColumn: "The used storage engine can't index column '%-.192s'", - ErWrongMrgTable: "Unable to open underlying table which is differently defined or of non-MyISAM type or doesn't exist", - ErDupUnique: "Can't write, because of unique constraint, to table '%-.192s'", - ErBlobKeyWithoutLength: "BLOB/TEXT column '%-.192s' used in key specification without a key length", - ErPrimaryCantHaveNull: "All parts of a PRIMARY KEY must be NOT NULL; if you need NULL in a key, use UNIQUE instead", - ErTooManyRows: "Result consisted of more than one row", - ErRequiresPrimaryKey: "This table type requires a primary key", - ErNoRaidCompiled: "This version of MySQL is not compiled with RAID support", - ErUpdateWithoutKeyInSafeMode: "You are using safe update mode and you tried to update a table without a WHERE that uses a KEY column", - ErKeyDoesNotExits: "Key '%-.192s' doesn't exist in table '%-.192s'", - ErCheckNoSuchTable: "Can't open table", - ErCheckNotImplemented: "The storage engine for the table doesn't support %s", - ErCantDoThisDuringAnTransaction: "You are not allowed to execute this command in a transaction", - ErErrorDuringCommit: "Got error %d during COMMIT", - ErErrorDuringRollback: "Got error %d during ROLLBACK", - ErErrorDuringFlushLogs: "Got error %d during FLUSHLOGS", - ErErrorDuringCheckpoint: "Got error %d during CHECKPOINT", - ErNewAbortingConnection: "Aborted connection %ld to db: '%-.192s' user: '%-.48s' host: '%-.64s' (%-.64s)", - ErDumpNotImplemented: "The storage engine for the table does not support binary table dump", - ErFlushMasterBinlogClosed: "Binlog closed, cannot RESET MASTER", - ErIndexRebuild: "Failed rebuilding the index of dumped table '%-.192s'", - ErMaster: "Error from master: '%-.64s'", - ErMasterNetRead: "Net error reading from master", - ErMasterNetWrite: "Net error writing to master", - ErFtMatchingKeyNotFound: "Can't find FULLTEXT index matching the column list", - ErLockOrActiveTransaction: "Can't execute the given command because you have active locked tables or an active transaction", - ErUnknownSystemVariable: "Unknown system variable '%-.64s'", - ErCrashedOnUsage: "Table '%-.192s' is marked as crashed and should be repaired", - ErCrashedOnRepair: "Table '%-.192s' is marked as crashed and last (automatic?) repair failed", - ErWarningNotCompleteRollback: "Some non-transactional changed tables couldn't be rolled back", - ErTransCacheFull: "Multi-statement transaction required more than 'maxBinlogCacheSize' bytes of storage; increase this mysqld variable and try again", - ErSlaveMustStop: "This operation cannot be performed with a running slave; run STOP SLAVE first", - ErSlaveNotRunning: "This operation requires a running slave; configure slave and do START SLAVE", - ErBadSlave: "The server is not configured as slave; fix in config file or with CHANGE MASTER TO", - ErMasterInfo: "Could not initialize master info structure; more error messages can be found in the MySQL error log", - ErSlaveThread: "Could not create slave thread; check system resources", - ErTooManyUserConnections: "User %-.64s already has more than 'maxUserConnections' active connections", - ErSetConstantsOnly: "You may only use constant expressions with SET", - ErLockWaitTimeout: "Lock wait timeout exceeded; try restarting transaction", - ErLockTableFull: "The total number of locks exceeds the lock table size", - ErReadOnlyTransaction: "Update locks cannot be acquired during a READ UNCOMMITTED transaction", - ErDropDbWithReadLock: "DROP DATABASE not allowed while thread is holding global read lock", - ErCreateDbWithReadLock: "CREATE DATABASE not allowed while thread is holding global read lock", - ErWrongArguments: "Incorrect arguments to %s", - ErNoPermissionToCreateUser: "'%-.48s'@'%-.64s' is not allowed to create new users", - ErUnionTablesInDifferentDir: "Incorrect table definition; all MERGE tables must be in the same database", - ErLockDeadlock: "Deadlock found when trying to get lock; try restarting transaction", - ErTableCantHandleFt: "The used table type doesn't support FULLTEXT indexes", - ErCannotAddForeign: "Cannot add foreign key constraint", - ErNoReferencedRow: "Cannot add or update a child row: a foreign key constraint fails", - ErRowIsReferenced: "Cannot delete or update a parent row: a foreign key constraint fails", - ErConnectToMaster: "Error connecting to master: %-.128s", - ErQueryOnMaster: "Error running query on master: %-.128s", - ErErrorWhenExecutingCommand: "Error when executing command %s: %-.128s", - ErWrongUsage: "Incorrect usage of %s and %s", - ErWrongNumberOfColumnsInSelect: "The used SELECT statements have a different number of columns", - ErCantUpdateWithReadlock: "Can't execute the query because you have a conflicting read lock", - ErMixingNotAllowed: "Mixing of transactional and non-transactional tables is disabled", - ErDupArgument: "Option '%s' used twice in statement", - ErUserLimitReached: "User '%-.64s' has exceeded the '%s' resource (current value: %ld)", - ErSpecificAccessDeniedError: "Access denied; you need (at least one of) the %-.128s privilege(s) for this operation", - ErLocalVariable: "Variable '%-.64s' is a SESSION variable and can't be used with SET GLOBAL", - ErGlobalVariable: "Variable '%-.64s' is a GLOBAL variable and should be set with SET GLOBAL", - ErNoDefault: "Variable '%-.64s' doesn't have a default value", - ErWrongValueForVar: "Variable '%-.64s' can't be set to the value of '%-.200s'", - ErWrongTypeForVar: "Incorrect argument type to variable '%-.64s'", - ErVarCantBeRead: "Variable '%-.64s' can only be set, not read", - ErCantUseOptionHere: "Incorrect usage/placement of '%s'", - ErNotSupportedYet: "This version of MySQL doesn't yet support '%s'", - ErMasterFatalErrorReadingBinlog: "Got fatal error %d from master when reading data from binary log: '%-.320s'", - ErSlaveIgnoredTable: "Slave SQL thread ignored the query because of replicate-*-table rules", - ErIncorrectGlobalLocalVar: "Variable '%-.192s' is a %s variable", - ErWrongFkDef: "Incorrect foreign key definition for '%-.192s': %s", - ErKeyRefDoNotMatchTableRef: "Key reference and table reference don't match", - ErOperandColumns: "Operand should contain %d column(s)", - ErSubqueryNo1Row: "Subquery returns more than 1 row", - ErUnknownStmtHandler: "Unknown prepared statement handler (%.*s) given to %s", - ErCorruptHelpDb: "Help database is corrupt or does not exist", - ErCyclicReference: "Cyclic reference on subqueries", - ErAutoConvert: "Converting column '%s' from %s to %s", - ErIllegalReference: "Reference '%-.64s' not supported (%s)", - ErDerivedMustHaveAlias: "Every derived table must have its own alias", - ErSelectReduced: "Select %u was reduced during optimization", - ErTablenameNotAllowedHere: "Table '%-.192s' from one of the SELECTs cannot be used in %-.32s", - ErNotSupportedAuthMode: "Client does not support authentication protocol requested by server; consider upgrading MySQL client", - ErSpatialCantHaveNull: "All parts of a SPATIAL index must be NOT NULL", - ErCollationCharsetMismatch: "COLLATION '%s' is not valid for CHARACTER SET '%s'", - ErSlaveWasRunning: "Slave is already running", - ErSlaveWasNotRunning: "Slave already has been stopped", - ErTooBigForUncompress: "Uncompressed data size too large; the maximum size is %d (probably, length of uncompressed data was corrupted)", - ErZlibZMemError: "ZLIB: Not enough memory", - ErZlibZBufError: "ZLIB: Not enough room in the output buffer (probably, length of uncompressed data was corrupted)", - ErZlibZDataError: "ZLIB: Input data corrupted", - ErCutValueGroupConcat: "Row %u was cut by GROUPCONCAT()", - ErWarnTooFewRecords: "Row %ld doesn't contain data for all columns", - ErWarnTooManyRecords: "Row %ld was truncated; it contained more data than there were input columns", - ErWarnNullToNotnull: "Column set to default value; NULL supplied to NOT NULL column '%s' at row %ld", - ErWarnDataOutOfRange: "Out of range value for column '%s' at row %ld", - WarnDataTruncated: "Data truncated for column '%s' at row %ld", - ErWarnUsingOtherHandler: "Using storage engine %s for table '%s'", - ErCantAggregate2collations: "Illegal mix of collations (%s,%s) and (%s,%s) for operation '%s'", - ErDropUser: "Cannot drop one or more of the requested users", - ErRevokeGrants: "Can't revoke all privileges for one or more of the requested users", - ErCantAggregate3collations: "Illegal mix of collations (%s,%s), (%s,%s), (%s,%s) for operation '%s'", - ErCantAggregateNcollations: "Illegal mix of collations for operation '%s'", - ErVariableIsNotStruct: "Variable '%-.64s' is not a variable component (can't be used as XXXX.variableName)", - ErUnknownCollation: "Unknown collation: '%-.64s'", - ErSlaveIgnoredSslParams: "SSL parameters in CHANGE MASTER are ignored because this MySQL slave was compiled without SSL support; they can be used later if MySQL slave with SSL is started", - ErServerIsInSecureAuthMode: "Server is running in --secure-auth mode, but '%s'@'%s' has a password in the old format; please change the password to the new format", - ErWarnFieldResolved: "Field or reference '%-.192s%s%-.192s%s%-.192s' of SELECT #%d was resolved in SELECT #%d", - ErBadSlaveUntilCond: "Incorrect parameter or combination of parameters for START SLAVE UNTIL", - ErMissingSkipSlave: "It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mysqld restart", - ErUntilCondIgnored: "SQL thread is not to be started so UNTIL options are ignored", - ErWrongNameForIndex: "Incorrect index name '%-.100s'", - ErWrongNameForCatalog: "Incorrect catalog name '%-.100s'", - ErWarnQcResize: "Query cache failed to set size %lu; new query cache size is %lu", - ErBadFtColumn: "Column '%-.192s' cannot be part of FULLTEXT index", - ErUnknownKeyCache: "Unknown key cache '%-.100s'", - ErWarnHostnameWontWork: "MySQL is started in --skip-name-resolve mode; you must restart it without this switch for this grant to work", - ErUnknownStorageEngine: "Unknown storage engine '%s'", - ErWarnDeprecatedSyntax: "'%s' is deprecated and will be removed in a future release. Please use %s instead", - ErNonUpdatableTable: "The target table %-.100s of the %s is not updatable", - ErFeatureDisabled: "The '%s' feature is disabled; you need MySQL built with '%s' to have it working", - ErOptionPreventsStatement: "The MySQL server is running with the %s option so it cannot execute this statement", - ErDuplicatedValueInType: "Column '%-.100s' has duplicated value '%-.64s' in %s", - ErTruncatedWrongValue: "Truncated incorrect %-.32s value: '%-.128s'", - ErTooMuchAutoTimestampCols: "Incorrect table definition; there can be only one TIMESTAMP column with CURRENTTIMESTAMP in DEFAULT or ON UPDATE clause", - ErInvalidOnUpdate: "Invalid ON UPDATE clause for '%-.192s' column", - ErUnsupportedPs: "This command is not supported in the prepared statement protocol yet", - ErGetErrmsg: "Got error %d '%-.100s' from %s", - ErGetTemporaryErrmsg: "Got temporary error %d '%-.100s' from %s", - ErUnknownTimeZone: "Unknown or incorrect time zone: '%-.64s'", - ErWarnInvalidTimestamp: "Invalid TIMESTAMP value in column '%s' at row %ld", - ErInvalidCharacterString: "Invalid %s character string: '%.64s'", - ErWarnAllowedPacketOverflowed: "Result of %s() was larger than maxAllowedPacket (%ld) - truncated", - ErConflictingDeclarations: "Conflicting declarations: '%s%s' and '%s%s'", - ErSpNoRecursiveCreate: "Can't create a %s from within another stored routine", - ErSpAlreadyExists: "%s %s already exists", - ErSpDoesNotExist: "%s %s does not exist", - ErSpDropFailed: "Failed to DROP %s %s", - ErSpStoreFailed: "Failed to CREATE %s %s", - ErSpLilabelMismatch: "%s with no matching label: %s", - ErSpLabelRedefine: "Redefining label %s", - ErSpLabelMismatch: "End-label %s without match", - ErSpUninitVar: "Referring to uninitialized variable %s", - ErSpBadselect: "PROCEDURE %s can't return a result set in the given context", - ErSpBadreturn: "RETURN is only allowed in a FUNCTION", - ErSpBadstatement: "%s is not allowed in stored procedures", - ErUpdateLogDeprecatedIgnored: "The update log is deprecated and replaced by the binary log; SET SQLLOGUPDATE has been ignored.", - ErUpdateLogDeprecatedTranslated: "The update log is deprecated and replaced by the binary log; SET SQLLOGUPDATE has been translated to SET SQLLOGBIN.", - ErQueryInterrupted: "Query execution was interrupted", - ErSpWrongNoOfArgs: "Incorrect number of arguments for %s %s; expected %u, got %u", - ErSpCondMismatch: "Undefined CONDITION: %s", - ErSpNoreturn: "No RETURN found in FUNCTION %s", - ErSpNoreturnend: "FUNCTION %s ended without RETURN", - ErSpBadCursorQuery: "Cursor statement must be a SELECT", - ErSpBadCursorSelect: "Cursor SELECT must not have INTO", - ErSpCursorMismatch: "Undefined CURSOR: %s", - ErSpCursorAlreadyOpen: "Cursor is already open", - ErSpCursorNotOpen: "Cursor is not open", - ErSpUndeclaredVar: "Undeclared variable: %s", - ErSpWrongNoOfFetchArgs: "Incorrect number of FETCH variables", - ErSpFetchNoData: "No data - zero rows fetched, selected, or processed", - ErSpDupParam: "Duplicate parameter: %s", - ErSpDupVar: "Duplicate variable: %s", - ErSpDupCond: "Duplicate condition: %s", - ErSpDupCurs: "Duplicate cursor: %s", - ErSpCantAlter: "Failed to ALTER %s %s", - ErSpSubselectNyi: "Subquery value not supported", - ErStmtNotAllowedInSfOrTrg: "%s is not allowed in stored function or trigger", - ErSpVarcondAfterCurshndlr: "Variable or condition declaration after cursor or handler declaration", - ErSpCursorAfterHandler: "Cursor declaration after handler declaration", - ErSpCaseNotFound: "Case not found for CASE statement", - ErFparserTooBigFile: "Configuration file '%-.192s' is too big", - ErFparserBadHeader: "Malformed file type header in file '%-.192s'", - ErFparserEOFInComment: "Unexpected end of file while parsing comment '%-.200s'", - ErFparserErrorInParameter: "Error while parsing parameter '%-.192s' (line: '%-.192s')", - ErFparserEOFInUnknownParameter: "Unexpected end of file while skipping unknown parameter '%-.192s'", - ErViewNoExplain: "EXPLAIN/SHOW can not be issued; lacking privileges for underlying table", - ErFrmUnknownType: "File '%-.192s' has unknown type '%-.64s' in its header", - ErWrongObject: "'%-.192s.%-.192s' is not %s", - ErNonupdateableColumn: "Column '%-.192s' is not updatable", - ErViewSelectDerived: "View's SELECT contains a subquery in the FROM clause", - ErViewSelectClause: "View's SELECT contains a '%s' clause", - ErViewSelectVariable: "View's SELECT contains a variable or parameter", - ErViewSelectTmptable: "View's SELECT refers to a temporary table '%-.192s'", - ErViewWrongList: "View's SELECT and view's field list have different column counts", - ErWarnViewMerge: "View merge algorithm can't be used here for now (assumed undefined algorithm)", - ErWarnViewWithoutKey: "View being updated does not have complete key of underlying table in it", - ErViewInvalid: "View '%-.192s.%-.192s' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them", - ErSpNoDropSp: "Can't drop or alter a %s from within another stored routine", - ErSpGotoInHndlr: "GOTO is not allowed in a stored procedure handler", - ErTrgAlreadyExists: "Trigger already exists", - ErTrgDoesNotExist: "Trigger does not exist", - ErTrgOnViewOrTempTable: "Trigger's '%-.192s' is view or temporary table", - ErTrgCantChangeRow: "Updating of %s row is not allowed in %strigger", - ErTrgNoSuchRowInTrg: "There is no %s row in %s trigger", - ErNoDefaultForField: "Field '%-.192s' doesn't have a default value", - ErDivisionByZero: "Division by 0", - ErTruncatedWrongValueForField: "Incorrect %-.32s value: '%-.128s' for column '%.192s' at row %ld", - ErIllegalValueForType: "Illegal %s '%-.192s' value found during parsing", - ErViewNonupdCheck: "CHECK OPTION on non-updatable view '%-.192s.%-.192s'", - ErViewCheckFailed: "CHECK OPTION failed '%-.192s.%-.192s'", - ErProcaccessDeniedError: "%-.16s command denied to user '%-.48s'@'%-.64s' for routine '%-.192s'", - ErRelayLogFail: "Failed purging old relay logs: %s", - ErPasswdLength: "Password hash should be a %d-digit hexadecimal number", - ErUnknownTargetBinlog: "Target log not found in binlog index", - ErIoErrLogIndexRead: "I/O error reading log index file", - ErBinlogPurgeProhibited: "Server configuration does not permit binlog purge", - ErFseekFail: "Failed on fseek()", - ErBinlogPurgeFatalErr: "Fatal error during log purge", - ErLogInUse: "A purgeable log is in use, will not purge", - ErLogPurgeUnknownErr: "Unknown error during log purge", - ErRelayLogInit: "Failed initializing relay log position: %s", - ErNoBinaryLogging: "You are not using binary logging", - ErReservedSyntax: "The '%-.64s' syntax is reserved for purposes internal to the MySQL server", - ErWsasFailed: "WSAStartup Failed", - ErDiffGroupsProc: "Can't handle procedures with different groups yet", - ErNoGroupForProc: "Select must have a group with this procedure", - ErOrderWithProc: "Can't use ORDER clause with this procedure", - ErLoggingProhibitChangingOf: "Binary logging and replication forbid changing the global server %s", - ErNoFileMapping: "Can't map file: %-.200s, errno: %d", - ErWrongMagic: "Wrong magic in %-.64s", - ErPsManyParam: "Prepared statement contains too many placeholders", - ErKeyPart0: "Key part '%-.192s' length cannot be 0", - ErViewChecksum: "View text checksum failed", - ErViewMultiupdate: "Can not modify more than one base table through a join view '%-.192s.%-.192s'", - ErViewNoInsertFieldList: "Can not insert into join view '%-.192s.%-.192s' without fields list", - ErViewDeleteMergeView: "Can not delete from join view '%-.192s.%-.192s'", - ErCannotUser: "Operation %s failed for %.256s", - ErXaerNota: "XAERNOTA: Unknown XID", - ErXaerInval: "XAERINVAL: Invalid arguments (or unsupported command)", - ErXaerRmfail: "XAERRMFAIL: The command cannot be executed when global transaction is in the %.64s state", - ErXaerOutside: "XAEROUTSIDE: Some work is done outside global transaction", - ErXaerRmerr: "XAERRMERR: Fatal error occurred in the transaction branch - check your data for consistency", - ErXaRbrollback: "XARBROLLBACK: Transaction branch was rolled back", - ErNonexistingProcGrant: "There is no such grant defined for user '%-.48s' on host '%-.64s' on routine '%-.192s'", - ErProcAutoGrantFail: "Failed to grant EXECUTE and ALTER ROUTINE privileges", - ErProcAutoRevokeFail: "Failed to revoke all privileges to dropped routine", - ErDataTooLong: "Data too long for column '%s' at row %ld", - ErSpBadSQLstate: "Bad SQLSTATE: '%s'", - ErStartup: "%s: ready for connections.\nVersion: '%s' socket: '%s' port: %d %s", - ErLoadFromFixedSizeRowsToVar: "Can't load value from file with fixed size rows to variable", - ErCantCreateUserWithGrant: "You are not allowed to create a user with GRANT", - ErWrongValueForType: "Incorrect %-.32s value: '%-.128s' for function %-.32s", - ErTableDefChanged: "Table definition has changed, please retry transaction", - ErSpDupHandler: "Duplicate handler declared in the same block", - ErSpNotVarArg: "OUT or INOUT argument %d for routine %s is not a variable or NEW pseudo-variable in BEFORE trigger", - ErSpNoRetset: "Not allowed to return a result set from a %s", - ErCantCreateGeometryObject: "Cannot get geometry object from data you send to the GEOMETRY field", - ErFailedRoutineBreakBinlog: "A routine failed and has neither NO SQL nor READS SQL DATA in its declaration and binary logging is enabled; if non-transactional tables were updated, the binary log will miss their changes", - ErBinlogUnsafeRoutine: "This function has none of DETERMINISTIC, NO SQL, or READS SQL DATA in its declaration and binary logging is enabled (you *might* want to use the less safe logBinTrustFunctionCreators variable)", - ErBinlogCreateRoutineNeedSuper: "You do not have the SUPER privilege and binary logging is enabled (you *might* want to use the less safe logBinTrustFunctionCreators variable)", - ErExecStmtWithOpenCursor: "You can't execute a prepared statement which has an open cursor associated with it. Reset the statement to re-execute it.", - ErStmtHasNoOpenCursor: "The statement (%lu) has no open cursor.", - ErCommitNotAllowedInSfOrTrg: "Explicit or implicit commit is not allowed in stored function or trigger.", - ErNoDefaultForViewField: "Field of view '%-.192s.%-.192s' underlying table doesn't have a default value", - ErSpNoRecursion: "Recursive stored functions and triggers are not allowed.", - ErTooBigScale: "Too big scale %d specified for column '%-.192s'. Maximum is %lu.", - ErTooBigPrecision: "Too big precision %d specified for column '%-.192s'. Maximum is %lu.", - ErMBiggerThanD: "For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '%-.192s').", - ErWrongLockOfSystemTable: "You can't combine write-locking of system tables with other tables or lock types", - ErConnectToForeignDataSource: "Unable to connect to foreign data source: %.64s", - ErQueryOnForeignDataSource: "There was a problem processing the query on the foreign data source. Data source error: %-.64s", - ErForeignDataSourceDoesntExist: "The foreign data source you are trying to reference does not exist. Data source error: %-.64s", - ErForeignDataStringInvalidCantCreate: "Can't create federated table. The data source connection string '%-.64s' is not in the correct format", - ErForeignDataStringInvalid: "The data source connection string '%-.64s' is not in the correct format", - ErCantCreateFederatedTable: "Can't create federated table. Foreign data src error: %-.64s", - ErTrgInWrongSchema: "Trigger in wrong schema", - ErStackOverrunNeedMore: "Thread stack overrun: %ld bytes used of a %ld byte stack, and %ld bytes needed. Use 'mysqld --threadStack=#' to specify a bigger stack.", - ErTooLongBody: "Routine body for '%-.100s' is too long", - ErWarnCantDropDefaultKeycache: "Cannot drop default keycache", - ErTooBigDisplaywidth: "Display width out of range for column '%-.192s' (max = %lu)", - ErXaerDupid: "XAERDUPID: The XID already exists", - ErDatetimeFunctionOverflow: "Datetime function: %-.32s field overflow", - ErCantUpdateUsedTableInSfOrTrg: "Can't update table '%-.192s' in stored function/trigger because it is already used by statement which invoked this stored function/trigger.", - ErViewPreventUpdate: "The definition of table '%-.192s' prevents operation %.192s on table '%-.192s'.", - ErPsNoRecursion: "The prepared statement contains a stored routine call that refers to that same statement. It's not allowed to execute a prepared statement in such a recursive manner", - ErSpCantSetAutocommit: "Not allowed to set autocommit from a stored function or trigger", - ErMalformedDefiner: "Definer is not fully qualified", - ErViewFrmNoUser: "View '%-.192s'.'%-.192s' has no definer information (old table format). Current user is used as definer. Please recreate the view!", - ErViewOtherUser: "You need the SUPER privilege for creation view with '%-.192s'@'%-.192s' definer", - ErNoSuchUser: "The user specified as a definer ('%-.64s'@'%-.64s') does not exist", - ErForbidSchemaChange: "Changing schema from '%-.192s' to '%-.192s' is not allowed.", - ErRowIsReferenced2: "Cannot delete or update a parent row: a foreign key constraint fails (%.192s)", - ErNoReferencedRow2: "Cannot add or update a child row: a foreign key constraint fails (%.192s)", - ErSpBadVarShadow: "Variable '%-.64s' must be quoted with `...`, or renamed", - ErTrgNoDefiner: "No definer attribute for trigger '%-.192s'.'%-.192s'. The trigger will be activated under the authorization of the invoker, which may have insufficient privileges. Please recreate the trigger.", - ErOldFileFormat: "'%-.192s' has an old format, you should re-create the '%s' object(s)", - ErSpRecursionLimit: "Recursive limit %d (as set by the maxSpRecursionDepth variable) was exceeded for routine %.192s", - ErSpProcTableCorrupt: "Failed to load routine %-.192s. The table mysql.proc is missing, corrupt, or contains bad data (internal code %d)", - ErSpWrongName: "Incorrect routine name '%-.192s'", - ErTableNeedsUpgrade: "Table upgrade required. Please do \"REPAIR TABLE `%-.32s`\"", - ErSpNoAggregate: "AGGREGATE is not supported for stored functions", - ErMaxPreparedStmtCountReached: "Can't create more than maxPreparedStmtCount statements (current value: %lu)", - ErViewRecursive: "`%-.192s`.`%-.192s` contains view recursion", - ErNonGroupingFieldUsed: "Non-grouping field '%-.192s' is used in %-.64s clause", - ErTableCantHandleSpkeys: "The used table type doesn't support SPATIAL indexes", - ErNoTriggersOnSystemSchema: "Triggers can not be created on system tables", - ErRemovedSpaces: "Leading spaces are removed from name '%s'", - ErAutoincReadFailed: "Failed to read auto-increment value from storage engine", - ErUsername: "user name", - ErHostname: "host name", - ErWrongStringLength: "String '%-.70s' is too long for %s (should be no longer than %d)", - ErNonInsertableTable: "The target table %-.100s of the %s is not insertable-into", - ErAdminWrongMrgTable: "Table '%-.64s' is differently defined or of non-MyISAM type or doesn't exist", - ErTooHighLevelOfNestingForSelect: "Too high level of nesting for select", - ErNameBecomesEmpty: "Name '%-.64s' has become ''", - ErAmbiguousFieldTerm: "First character of the FIELDS TERMINATED string is ambiguous; please use non-optional and non-empty FIELDS ENCLOSED BY", - ErForeignServerExists: "The foreign server, %s, you are trying to create already exists.", - ErForeignServerDoesntExist: "The foreign server name you are trying to reference does not exist. Data source error: %-.64s", - ErIllegalHaCreateOption: "Table storage engine '%-.64s' does not support the create option '%.64s'", - ErPartitionRequiresValuesError: "Syntax error: %-.64s PARTITIONING requires definition of VALUES %-.64s for each partition", - ErPartitionWrongValuesError: "Only %-.64s PARTITIONING can use VALUES %-.64s in partition definition", - ErPartitionMaxvalueError: "MAXVALUE can only be used in last partition definition", - ErPartitionSubpartitionError: "Subpartitions can only be hash partitions and by key", - ErPartitionSubpartMixError: "Must define subpartitions on all partitions if on one partition", - ErPartitionWrongNoPartError: "Wrong number of partitions defined, mismatch with previous setting", - ErPartitionWrongNoSubpartError: "Wrong number of subpartitions defined, mismatch with previous setting", - ErWrongExprInPartitionFuncError: "Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed", - ErNoConstExprInRangeOrListError: "Expression in RANGE/LIST VALUES must be constant", - ErFieldNotFoundPartError: "Field in list of fields for partition function not found in table", - ErListOfFieldsOnlyInHashError: "List of fields is only allowed in KEY partitions", - ErInconsistentPartitionInfoError: "The partition info in the frm file is not consistent with what can be written into the frm file", - ErPartitionFuncNotAllowedError: "The %-.192s function returns the wrong type", - ErPartitionsMustBeDefinedError: "For %-.64s partitions each partition must be defined", - ErRangeNotIncreasingError: "VALUES LESS THAN value must be strictly increasing for each partition", - ErInconsistentTypeOfFunctionsError: "VALUES value must be of same type as partition function", - ErMultipleDefConstInListPartError: "Multiple definition of same constant in list partitioning", - ErPartitionEntryError: "Partitioning can not be used stand-alone in query", - ErMixHandlerError: "The mix of handlers in the partitions is not allowed in this version of MySQL", - ErPartitionNotDefinedError: "For the partitioned engine it is necessary to define all %-.64s", - ErTooManyPartitionsError: "Too many partitions (including subpartitions) were defined", - ErSubpartitionError: "It is only possible to mix RANGE/LIST partitioning with HASH/KEY partitioning for subpartitioning", - ErCantCreateHandlerFile: "Failed to create specific handler file", - ErBlobFieldInPartFuncError: "A BLOB field is not allowed in partition function", - ErUniqueKeyNeedAllFieldsInPf: "A %-.192s must include all columns in the table's partitioning function", - ErNoPartsError: "Number of %-.64s = 0 is not an allowed value", - ErPartitionMgmtOnNonpartitioned: "Partition management on a not partitioned table is not possible", - ErForeignKeyOnPartitioned: "Foreign key clause is not yet supported in conjunction with partitioning", - ErDropPartitionNonExistent: "Error in list of partitions to %-.64s", - ErDropLastPartition: "Cannot remove all partitions, use DROP TABLE instead", - ErCoalesceOnlyOnHashPartition: "COALESCE PARTITION can only be used on HASH/KEY partitions", - ErReorgHashOnlyOnSameNo: "REORGANIZE PARTITION can only be used to reorganize partitions not to change their numbers", - ErReorgNoParamError: "REORGANIZE PARTITION without parameters can only be used on auto-partitioned tables using HASH PARTITIONs", - ErOnlyOnRangeListPartition: "%-.64s PARTITION can only be used on RANGE/LIST partitions", - ErAddPartitionSubpartError: "Trying to Add partition(s) with wrong number of subpartitions", - ErAddPartitionNoNewPartition: "At least one partition must be added", - ErCoalescePartitionNoPartition: "At least one partition must be coalesced", - ErReorgPartitionNotExist: "More partitions to reorganize than there are partitions", - ErSameNamePartition: "Duplicate partition name %-.192s", - ErNoBinlogError: "It is not allowed to shut off binlog on this command", - ErConsecutiveReorgPartitions: "When reorganizing a set of partitions they must be in consecutive order", - ErReorgOutsideRange: "Reorganize of range partitions cannot change total ranges except for last partition where it can extend the range", - ErPartitionFunctionFailure: "Partition function not supported in this version for this handler", - ErPartStateError: "Partition state cannot be defined from CREATE/ALTER TABLE", - ErLimitedPartRange: "The %-.64s handler only supports 32 bit integers in VALUES", - ErPluginIsNotLoaded: "Plugin '%-.192s' is not loaded", - ErWrongValue: "Incorrect %-.32s value: '%-.128s'", - ErNoPartitionForGivenValue: "Table has no partition for value %-.64s", - ErFilegroupOptionOnlyOnce: "It is not allowed to specify %s more than once", - ErCreateFilegroupFailed: "Failed to create %s", - ErDropFilegroupFailed: "Failed to drop %s", - ErTablespaceAutoExtendError: "The handler doesn't support autoextend of tablespaces", - ErWrongSizeNumber: "A size parameter was incorrectly specified, either number or on the form 10M", - ErSizeOverflowError: "The size number was correct but we don't allow the digit part to be more than 2 billion", - ErAlterFilegroupFailed: "Failed to alter: %s", - ErBinlogRowLoggingFailed: "Writing one row to the row-based binary log failed", - ErBinlogRowWrongTableDef: "Table definition on master and slave does not match: %s", - ErBinlogRowRbrToSbr: "Slave running with --log-slave-updates must use row-based binary logging to be able to replicate row-based binary log events", - ErEventAlreadyExists: "Event '%-.192s' already exists", - ErEventStoreFailed: "Failed to store event %s. Error code %d from storage engine.", - ErEventDoesNotExist: "Unknown event '%-.192s'", - ErEventCantAlter: "Failed to alter event '%-.192s'", - ErEventDropFailed: "Failed to drop %s", - ErEventIntervalNotPositiveOrTooBig: "INTERVAL is either not positive or too big", - ErEventEndsBeforeStarts: "ENDS is either invalid or before STARTS", - ErEventExecTimeInThePast: "Event execution time is in the past. Event has been disabled", - ErEventOpenTableFailed: "Failed to open mysql.event", - ErEventNeitherMExprNorMAt: "No datetime expression provided", - ErObsoleteColCountDoesntMatchCorrupted: "Column count of mysql.%s is wrong. Expected %d, found %d. The table is probably corrupted", - ErObsoleteCannotLoadFromTable: "Cannot load from mysql.%s. The table is probably corrupted", - ErEventCannotDelete: "Failed to delete the event from mysql.event", - ErEventCompileError: "Error during compilation of event's body", - ErEventSameName: "Same old and new event name", - ErEventDataTooLong: "Data for column '%s' too long", - ErDropIndexFk: "Cannot drop index '%-.192s': needed in a foreign key constraint", - ErWarnDeprecatedSyntaxWithVer: "The syntax '%s' is deprecated and will be removed in MySQL %s. Please use %s instead", - ErCantWriteLockLogTable: "You can't write-lock a log table. Only read access is possible", - ErCantLockLogTable: "You can't use locks with log tables.", - ErForeignDuplicateKeyOldUnused: "Upholding foreign key constraints for table '%.192s', entry '%-.192s', key %d would lead to a duplicate entry", - ErColCountDoesntMatchPleaseUpdate: "Column count of mysql.%s is wrong. Expected %d, found %d. Created with MySQL %d, now running %d. Please use mysqlUpgrade to fix this error.", - ErTempTablePreventsSwitchOutOfRbr: "Cannot switch out of the row-based binary log format when the session has open temporary tables", - ErStoredFunctionPreventsSwitchBinlogFormat: "Cannot change the binary logging format inside a stored function or trigger", - ErNdbCantSwitchBinlogFormat: "The NDB cluster engine does not support changing the binlog format on the fly yet", - ErPartitionNoTemporary: "Cannot create temporary table with partitions", - ErPartitionConstDomainError: "Partition constant is out of partition function domain", - ErPartitionFunctionIsNotAllowed: "This partition function is not allowed", - ErDdlLogError: "Error in DDL log", - ErNullInValuesLessThan: "Not allowed to use NULL value in VALUES LESS THAN", - ErWrongPartitionName: "Incorrect partition name", - ErCantChangeTxCharacteristics: "Transaction characteristics can't be changed while a transaction is in progress", - ErDupEntryAutoincrementCase: "ALTER TABLE causes autoIncrement resequencing, resulting in duplicate entry '%-.192s' for key '%-.192s'", - ErEventModifyQueueError: "Internal scheduler error %d", - ErEventSetVarError: "Error during starting/stopping of the scheduler. Error code %u", - ErPartitionMergeError: "Engine cannot be used in partitioned tables", - ErCantActivateLog: "Cannot activate '%-.64s' log", - ErRbrNotAvailable: "The server was not built with row-based replication", - ErBase64DecodeError: "Decoding of base64 string failed", - ErEventRecursionForbidden: "Recursion of EVENT DDL statements is forbidden when body is present", - ErEventsDbError: "Cannot proceed because system tables used by Event Scheduler were found damaged at server start", - ErOnlyIntegersAllowed: "Only integers allowed as number here", - ErUnsuportedLogEngine: "This storage engine cannot be used for log tables\"", - ErBadLogStatement: "You cannot '%s' a log table if logging is enabled", - ErCantRenameLogTable: "Cannot rename '%s'. When logging enabled, rename to/from log table must rename two tables: the log table to an archive table and another table back to '%s'", - ErWrongParamcountToNativeFct: "Incorrect parameter count in the call to native function '%-.192s'", - ErWrongParametersToNativeFct: "Incorrect parameters in the call to native function '%-.192s'", - ErWrongParametersToStoredFct: "Incorrect parameters in the call to stored function '%-.192s'", - ErNativeFctNameCollision: "This function '%-.192s' has the same name as a native function", - ErDupEntryWithKeyName: "Duplicate entry '%-.64s' for key '%-.192s'", - ErBinlogPurgeEmfile: "Too many files opened, please execute the command again", - ErEventCannotCreateInThePast: "Event execution time is in the past and ON COMPLETION NOT PRESERVE is set. The event was dropped immediately after creation.", - ErEventCannotAlterInThePast: "Event execution time is in the past and ON COMPLETION NOT PRESERVE is set. The event was not changed. Specify a time in the future.", - ErSlaveIncident: "The incident %s occured on the master. Message: %-.64s", - ErNoPartitionForGivenValueSilent: "Table has no partition for some existing values", - ErBinlogUnsafeStatement: "Unsafe statement written to the binary log using statement format since BINLOGFORMAT = STATEMENT. %s", - ErSlaveFatalError: "Fatal error: %s", - ErSlaveRelayLogReadFailure: "Relay log read failure: %s", - ErSlaveRelayLogWriteFailure: "Relay log write failure: %s", - ErSlaveCreateEventFailure: "Failed to create %s", - ErSlaveMasterComFailure: "Master command %s failed: %s", - ErBinlogLoggingImpossible: "Binary logging not possible. Message: %s", - ErViewNoCreationCtx: "View `%-.64s`.`%-.64s` has no creation context", - ErViewInvalidCreationCtx: "Creation context of view `%-.64s`.`%-.64s' is invalid", - ErSrInvalidCreationCtx: "Creation context of stored routine `%-.64s`.`%-.64s` is invalid", - ErTrgCorruptedFile: "Corrupted TRG file for table `%-.64s`.`%-.64s`", - ErTrgNoCreationCtx: "Triggers for table `%-.64s`.`%-.64s` have no creation context", - ErTrgInvalidCreationCtx: "Trigger creation context of table `%-.64s`.`%-.64s` is invalid", - ErEventInvalidCreationCtx: "Creation context of event `%-.64s`.`%-.64s` is invalid", - ErTrgCantOpenTable: "Cannot open table for trigger `%-.64s`.`%-.64s`", - ErCantCreateSroutine: "Cannot create stored routine `%-.64s`. Check warnings", - ErNeverUsed: "Ambiguous slave modes combination. %s", - ErNoFormatDescriptionEventBeforeBinlogStatement: "The BINLOG statement of type `%s` was not preceded by a format description BINLOG statement.", - ErSlaveCorruptEvent: "Corrupted replication event was detected", - ErLoadDataInvalidColumn: "Invalid column reference (%-.64s) in LOAD DATA", - ErLogPurgeNoFile: "Being purged log %s was not found", - ErXaRbtimeout: "XARBTIMEOUT: Transaction branch was rolled back: took too long", - ErXaRbdeadlock: "XARBDEADLOCK: Transaction branch was rolled back: deadlock was detected", - ErNeedReprepare: "Prepared statement needs to be re-prepared", - ErDelayedNotSupported: "DELAYED option not supported for table '%-.192s'", - WarnNoMasterInfo: "The master info structure does not exist", - WarnOptionIgnored: "<%-.64s> option ignored", - WarnPluginDeleteBuiltin: "Built-in plugins cannot be deleted", - WarnPluginBusy: "Plugin is busy and will be uninstalled on shutdown", - ErVariableIsReadonly: "%s variable '%s' is read-only. Use SET %s to assign the value", - ErWarnEngineTransactionRollback: "Storage engine %s does not support rollback for this statement. Transaction rolled back and must be restarted", - ErSlaveHeartbeatFailure: "Unexpected master's heartbeat data: %s", - ErSlaveHeartbeatValueOutOfRange: "The requested value for the heartbeat period is either negative or exceeds the maximum allowed (%s seconds).", - ErNdbReplicationSchemaError: "Bad schema for mysql.ndbReplication table. Message: %-.64s", - ErConflictFnParseError: "Error in parsing conflict function. Message: %-.64s", - ErExceptionsWriteError: "Write to exceptions table failed. Message: %-.128s\"", - ErTooLongTableComment: "Comment for table '%-.64s' is too long (max = %lu)", - ErTooLongFieldComment: "Comment for field '%-.64s' is too long (max = %lu)", - ErFuncInexistentNameCollision: "FUNCTION %s does not exist. Check the 'Function Name Parsing and Resolution' section in the Reference Manual", - ErDatabaseName: "Database", - ErTableName: "Table", - ErPartitionName: "Partition", - ErSubpartitionName: "Subpartition", - ErTemporaryName: "Temporary", - ErRenamedName: "Renamed", - ErTooManyConcurrentTrxs: "Too many active concurrent transactions", - WarnNonASCIISeparatorNotImplemented: "Non-ASCII separator arguments are not fully supported", - ErDebugSyncTimeout: "debug sync point wait timed out", - ErDebugSyncHitLimit: "debug sync point hit limit reached", - ErDupSignalSet: "Duplicate condition information item '%s'", - ErSignalWarn: "Unhandled user-defined warning condition", - ErSignalNotFound: "Unhandled user-defined not found condition", - ErSignalException: "Unhandled user-defined exception condition", - ErResignalWithoutActiveHandler: "RESIGNAL when handler not active", - ErSignalBadConditionType: "SIGNAL/RESIGNAL can only use a CONDITION defined with SQLSTATE", - WarnCondItemTruncated: "Data truncated for condition item '%s'", - ErCondItemTooLong: "Data too long for condition item '%s'", - ErUnknownLocale: "Unknown locale: '%-.64s'", - ErSlaveIgnoreServerIds: "The requested server id %d clashes with the slave startup option --replicate-same-server-id", - ErQueryCacheDisabled: "Query cache is disabled; restart the server with queryCacheType=1 to enable it", - ErSameNamePartitionField: "Duplicate partition field name '%-.192s'", - ErPartitionColumnListError: "Inconsistency in usage of column lists for partitioning", - ErWrongTypeColumnValueError: "Partition column values of incorrect type", - ErTooManyPartitionFuncFieldsError: "Too many fields in '%-.192s'", - ErMaxvalueInValuesIn: "Cannot use MAXVALUE as value in VALUES IN", - ErTooManyValuesError: "Cannot have more than one value for this type of %-.64s partitioning", - ErRowSinglePartitionFieldError: "Row expressions in VALUES IN only allowed for multi-field column partitioning", - ErFieldTypeNotAllowedAsPartitionField: "Field '%-.192s' is of a not allowed type for this type of partitioning", - ErPartitionFieldsTooLong: "The total length of the partitioning fields is too large", - ErBinlogRowEngineAndStmtEngine: "Cannot execute statement: impossible to write to binary log since both row-incapable engines and statement-incapable engines are involved.", - ErBinlogRowModeAndStmtEngine: "Cannot execute statement: impossible to write to binary log since BINLOGFORMAT = ROW and at least one table uses a storage engine limited to statement-based logging.", - ErBinlogUnsafeAndStmtEngine: "Cannot execute statement: impossible to write to binary log since statement is unsafe, storage engine is limited to statement-based logging, and BINLOGFORMAT = MIXED. %s", - ErBinlogRowInjectionAndStmtEngine: "Cannot execute statement: impossible to write to binary log since statement is in row format and at least one table uses a storage engine limited to statement-based logging.", - ErBinlogStmtModeAndRowEngine: "Cannot execute statement: impossible to write to binary log since BINLOGFORMAT = STATEMENT and at least one table uses a storage engine limited to row-based logging.%s", - ErBinlogRowInjectionAndStmtMode: "Cannot execute statement: impossible to write to binary log since statement is in row format and BINLOGFORMAT = STATEMENT.", - ErBinlogMultipleEnginesAndSelfLoggingEngine: "Cannot execute statement: impossible to write to binary log since more than one engine is involved and at least one engine is self-logging.", - ErBinlogUnsafeLimit: "The statement is unsafe because it uses a LIMIT clause. This is unsafe because the set of rows included cannot be predicted.", - ErBinlogUnsafeInsertDelayed: "The statement is unsafe because it uses INSERT DELAYED. This is unsafe because the times when rows are inserted cannot be predicted.", - ErBinlogUnsafeSystemTable: "The statement is unsafe because it uses the general log, slow query log, or performanceSchema table(s). This is unsafe because system tables may differ on slaves.", - ErBinlogUnsafeAutoincColumns: "Statement is unsafe because it invokes a trigger or a stored function that inserts into an AUTOINCREMENT column. Inserted values cannot be logged correctly.", - ErBinlogUnsafeUdf: "Statement is unsafe because it uses a UDF which may not return the same value on the slave.", - ErBinlogUnsafeSystemVariable: "Statement is unsafe because it uses a system variable that may have a different value on the slave.", - ErBinlogUnsafeSystemFunction: "Statement is unsafe because it uses a system function that may return a different value on the slave.", - ErBinlogUnsafeNontransAfterTrans: "Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction.", - ErMessageAndStatement: "%s Statement: %s", - ErSlaveConversionFailed: "Column %d of table '%-.192s.%-.192s' cannot be converted from type '%-.32s' to type '%-.32s'", - ErSlaveCantCreateConversion: "Can't create conversion table for table '%-.192s.%-.192s'", - ErInsideTransactionPreventsSwitchBinlogFormat: "Cannot modify @@session.binlogFormat inside a transaction", - ErPathLength: "The path specified for %.64s is too long.", - ErWarnDeprecatedSyntaxNoReplacement: "'%s' is deprecated and will be removed in a future release.", - ErWrongNativeTableStructure: "Native table '%-.64s'.'%-.64s' has the wrong structure", - ErWrongPerfschemaUsage: "Invalid performanceSchema usage.", - ErWarnISSkippedTable: "Table '%s'.'%s' was skipped since its definition is being modified by concurrent DDL statement", - ErInsideTransactionPreventsSwitchBinlogDirect: "Cannot modify @@session.binlogDirectNonTransactionalUpdates inside a transaction", - ErStoredFunctionPreventsSwitchBinlogDirect: "Cannot change the binlog direct flag inside a stored function or trigger", - ErSpatialMustHaveGeomCol: "A SPATIAL index may only contain a geometrical type column", - ErTooLongIndexComment: "Comment for index '%-.64s' is too long (max = %lu)", - ErLockAborted: "Wait on a lock was aborted due to a pending exclusive lock", - ErDataOutOfRange: "%s value is out of range in '%s'", - ErWrongSpvarTypeInLimit: "A variable of a non-integer based type in LIMIT clause", - ErBinlogUnsafeMultipleEnginesAndSelfLoggingEngine: "Mixing self-logging and non-self-logging engines in a statement is unsafe.", - ErBinlogUnsafeMixedStatement: "Statement accesses nontransactional table as well as transactional or temporary table, and writes to any of them.", - ErInsideTransactionPreventsSwitchSQLLogBin: "Cannot modify @@session.sqlLogBin inside a transaction", - ErStoredFunctionPreventsSwitchSQLLogBin: "Cannot change the sqlLogBin inside a stored function or trigger", - ErFailedReadFromParFile: "Failed to read from the .par file", - ErValuesIsNotIntTypeError: "VALUES value for partition '%-.64s' must have type INT", - ErAccessDeniedNoPasswordError: "Access denied for user '%-.48s'@'%-.64s'", - ErSetPasswordAuthPlugin: "SET PASSWORD has no significance for users authenticating via plugins", - ErGrantPluginUserExists: "GRANT with IDENTIFIED WITH is illegal because the user %-.*s already exists", - ErTruncateIllegalFk: "Cannot truncate a table referenced in a foreign key constraint (%.192s)", - ErPluginIsPermanent: "Plugin '%s' is forcePlusPermanent and can not be unloaded", - ErSlaveHeartbeatValueOutOfRangeMin: "The requested value for the heartbeat period is less than 1 millisecond. The value is reset to 0, meaning that heartbeating will effectively be disabled.", - ErSlaveHeartbeatValueOutOfRangeMax: "The requested value for the heartbeat period exceeds the value of `slaveNetTimeout' seconds. A sensible value for the period should be less than the timeout.", - ErStmtCacheFull: "Multi-row statements required more than 'maxBinlogStmtCacheSize' bytes of storage; increase this mysqld variable and try again", - ErMultiUpdateKeyConflict: "Primary key/partition key update is not allowed since the table is updated both as '%-.192s' and '%-.192s'.", - ErTableNeedsRebuild: "Table rebuild required. Please do \"ALTER TABLE `%-.32s` FORCE\" or dump/reload to fix it!", - WarnOptionBelowLimit: "The value of '%s' should be no less than the value of '%s'", - ErIndexColumnTooLong: "Index column size too large. The maximum column size is %lu bytes.", - ErErrorInTriggerBody: "Trigger '%-.64s' has an error in its body: '%-.256s'", - ErErrorInUnknownTriggerBody: "Unknown trigger has an error in its body: '%-.256s'", - ErIndexCorrupt: "Index %s is corrupted", - ErUndoRecordTooBig: "Undo log record is too big.", - ErBinlogUnsafeInsertIgnoreSelect: "INSERT IGNORE... SELECT is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are ignored. This order cannot be predicted and may differ on master and the slave.", - ErBinlogUnsafeInsertSelectUpdate: "INSERT... SELECT... ON DUPLICATE KEY UPDATE is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are updated. This order cannot be predicted and may differ on master and the slave.", - ErBinlogUnsafeReplaceSelect: "REPLACE... SELECT is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are replaced. This order cannot be predicted and may differ on master and the slave.", - ErBinlogUnsafeCreateIgnoreSelect: "CREATE... IGNORE SELECT is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are ignored. This order cannot be predicted and may differ on master and the slave.", - ErBinlogUnsafeCreateReplaceSelect: "CREATE... REPLACE SELECT is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are replaced. This order cannot be predicted and may differ on master and the slave.", - ErBinlogUnsafeUpdateIgnore: "UPDATE IGNORE is unsafe because the order in which rows are updated determines which (if any) rows are ignored. This order cannot be predicted and may differ on master and the slave.", - ErPluginNoUninstall: "Plugin '%s' is marked as not dynamically uninstallable. You have to stop the server to uninstall it.", - ErPluginNoInstall: "Plugin '%s' is marked as not dynamically installable. You have to stop the server to install it.", - ErBinlogUnsafeWriteAutoincSelect: "Statements writing to a table with an auto-increment column after selecting from another table are unsafe because the order in which rows are retrieved determines what (if any) rows will be written. This order cannot be predicted and may differ on master and the slave.", - ErBinlogUnsafeCreateSelectAutoinc: "CREATE TABLE... SELECT... on a table with an auto-increment column is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are inserted. This order cannot be predicted and may differ on master and the slave.", - ErBinlogUnsafeInsertTwoKeys: "INSERT... ON DUPLICATE KEY UPDATE on a table with more than one UNIQUE KEY is unsafe", - ErTableInFkCheck: "Table is being used in foreign key check.", - ErUnsupportedEngine: "Storage engine '%s' does not support system tables. [%s.%s]", - ErBinlogUnsafeAutoincNotFirst: "INSERT into autoincrement field which is not the first part in the composed primary key is unsafe.", - ErCannotLoadFromTableV2: "Cannot load from %s.%s. The table is probably corrupted", - ErMasterDelayValueOutOfRange: "The requested value %u for the master delay exceeds the maximum %u", - ErOnlyFdAndRbrEventsAllowedInBinlogStatement: "Only FormatDescriptionLogEvent and row events are allowed in BINLOG statements (but %s was provided)", - ErPartitionExchangeDifferentOption: "Non matching attribute '%-.64s' between partition and table", - ErPartitionExchangePartTable: "Table to exchange with partition is partitioned: '%-.64s'", - ErPartitionExchangeTempTable: "Table to exchange with partition is temporary: '%-.64s'", - ErPartitionInsteadOfSubpartition: "Subpartitioned table, use subpartition instead of partition", - ErUnknownPartition: "Unknown partition '%-.64s' in table '%-.64s'", - ErTablesDifferentMetadata: "Tables have different definitions", - ErRowDoesNotMatchPartition: "Found a row that does not match the partition", - ErBinlogCacheSizeGreaterThanMax: "Option binlogCacheSize (%lu) is greater than maxBinlogCacheSize (%lu); setting binlogCacheSize equal to maxBinlogCacheSize.", - ErWarnIndexNotApplicable: "Cannot use %-.64s access on index '%-.64s' due to type or collation conversion on field '%-.64s'", - ErPartitionExchangeForeignKey: "Table to exchange with partition has foreign key references: '%-.64s'", - ErNoSuchKeyValue: "Key value '%-.192s' was not found in table '%-.192s.%-.192s'", - ErRplInfoDataTooLong: "Data for column '%s' too long", - ErNetworkReadEventChecksumFailure: "Replication event checksum verification failed while reading from network.", - ErBinlogReadEventChecksumFailure: "Replication event checksum verification failed while reading from a log file.", - ErBinlogStmtCacheSizeGreaterThanMax: "Option binlogStmtCacheSize (%lu) is greater than maxBinlogStmtCacheSize (%lu); setting binlogStmtCacheSize equal to maxBinlogStmtCacheSize.", - ErCantUpdateTableInCreateTableSelect: "Can't update table '%-.192s' while '%-.192s' is being created.", - ErPartitionClauseOnNonpartitioned: "PARTITION () clause on non partitioned table", - ErRowDoesNotMatchGivenPartitionSet: "Found a row not matching the given partition set", - ErNoSuchPartitionunused: "partition '%-.64s' doesn't exist", - ErChangeRplInfoRepositoryFailure: "Failure while changing the type of replication repository: %s.", - ErWarningNotCompleteRollbackWithCreatedTempTable: "The creation of some temporary tables could not be rolled back.", - ErWarningNotCompleteRollbackWithDroppedTempTable: "Some temporary tables were dropped, but these operations could not be rolled back.", - ErMtsFeatureIsNotSupported: "%s is not supported in multi-threaded slave mode. %s", - ErMtsUpdatedDbsGreaterMax: "The number of modified databases exceeds the maximum %d; the database names will not be included in the replication event metadata.", - ErMtsCantParallel: "Cannot execute the current event group in the parallel mode. Encountered event %s, relay-log name %s, position %s which prevents execution of this event group in parallel mode. Reason: %s.", - ErMtsInconsistentData: "%s", - ErFulltextNotSupportedWithPartitioning: "FULLTEXT index is not supported for partitioned tables.", - ErDaInvalidConditionNumber: "Invalid condition number", - ErInsecurePlainText: "Sending passwords in plain text without SSL/TLS is extremely insecure.", - ErInsecureChangeMaster: "Storing MySQL user name or password information in the master.info repository is not secure and is therefore not recommended. Please see the MySQL Manual for more about this issue and possible alternatives.", - ErForeignDuplicateKeyWithChildInfo: "Foreign key constraint for table '%.192s', record '%-.192s' would lead to a duplicate entry in table '%.192s', key '%.192s'", - ErForeignDuplicateKeyWithoutChildInfo: "Foreign key constraint for table '%.192s', record '%-.192s' would lead to a duplicate entry in a child table", - ErSQLthreadWithSecureSlave: "Setting authentication options is not possible when only the Slave SQL Thread is being started.", - ErTableHasNoFt: "The table does not have FULLTEXT index to support this query", - ErVariableNotSettableInSfOrTrigger: "The system variable %.200s cannot be set in stored functions or triggers.", - ErVariableNotSettableInTransaction: "The system variable %.200s cannot be set when there is an ongoing transaction.", - ErGtidNextIsNotInGtidNextList: "The system variable @@SESSION.GTIDNEXT has the value %.200s, which is not listed in @@SESSION.GTIDNEXTLIST.", - ErCantChangeGtidNextInTransactionWhenGtidNextListIsNull: "When @@SESSION.GTIDNEXTLIST == NULL, the system variable @@SESSION.GTIDNEXT cannot change inside a transaction.", - ErSetStatementCannotInvokeFunction: "The statement 'SET %.200s' cannot invoke a stored function.", - ErGtidNextCantBeAutomaticIfGtidNextListIsNonNull: "The system variable @@SESSION.GTIDNEXT cannot be 'AUTOMATIC' when @@SESSION.GTIDNEXTLIST is non-NULL.", - ErSkippingLoggedTransaction: "Skipping transaction %.200s because it has already been executed and logged.", - ErMalformedGtidSetSpecification: "Malformed GTID set specification '%.200s'.", - ErMalformedGtidSetEncoding: "Malformed GTID set encoding.", - ErMalformedGtidSpecification: "Malformed GTID specification '%.200s'.", - ErGnoExhausted: "Impossible to generate Global Transaction Identifier: the integer component reached the maximal value. Restart the server with a new serverUuid.", - ErBadSlaveAutoPosition: "Parameters MASTERLOGFILE, MASTERLOGPOS, RELAYLOGFILE and RELAYLOGPOS cannot be set when MASTERAUTOPOSITION is active.", - ErAutoPositionRequiresGtidModeOn: "CHANGE MASTER TO MASTERAUTOPOSITION = 1 can only be executed when @@GLOBAL.GTIDMODE = ON.", - ErCantDoImplicitCommitInTrxWhenGtidNextIsSet: "Cannot execute statements with implicit commit inside a transaction when @@SESSION.GTIDNEXT != AUTOMATIC or @@SESSION.GTIDNEXTLIST != NULL.", - ErGtidMode2Or3RequiresEnforceGtidConsistencyOn: "@@GLOBAL.GTIDMODE = ON or UPGRADESTEP2 requires @@GLOBAL.ENFORCEGTIDCONSISTENCY = 1.", - ErGtidModeRequiresBinlog: "@@GLOBAL.GTIDMODE = ON or UPGRADESTEP1 or UPGRADESTEP2 requires --log-bin and --log-slave-updates.", - ErCantSetGtidNextToGtidWhenGtidModeIsOff: "@@SESSION.GTIDNEXT cannot be set to UUID:NUMBER when @@GLOBAL.GTIDMODE = OFF.", - ErCantSetGtidNextToAnonymousWhenGtidModeIsOn: "@@SESSION.GTIDNEXT cannot be set to ANONYMOUS when @@GLOBAL.GTIDMODE = ON.", - ErCantSetGtidNextListToNonNullWhenGtidModeIsOff: "@@SESSION.GTIDNEXTLIST cannot be set to a non-NULL value when @@GLOBAL.GTIDMODE = OFF.", - ErFoundGtidEventWhenGtidModeIsOff: "Found a GtidLogEvent or PreviousGtidsLogEvent when @@GLOBAL.GTIDMODE = OFF.", - ErGtidUnsafeNonTransactionalTable: "When @@GLOBAL.ENFORCEGTIDCONSISTENCY = 1, updates to non-transactional tables can only be done in either autocommitted statements or single-statement transactions, and never in the same statement as updates to transactional tables.", - ErGtidUnsafeCreateSelect: "CREATE TABLE ... SELECT is forbidden when @@GLOBAL.ENFORCEGTIDCONSISTENCY = 1.", - ErGtidUnsafeCreateDropTemporaryTableInTransaction: "When @@GLOBAL.ENFORCEGTIDCONSISTENCY = 1, the statements CREATE TEMPORARY TABLE and DROP TEMPORARY TABLE can be executed in a non-transactional context only, and require that AUTOCOMMIT = 1.", - ErGtidModeCanOnlyChangeOneStepAtATime: "The value of @@GLOBAL.GTIDMODE can only change one step at a time: OFF <-> UPGRADESTEP1 <-> UPGRADESTEP2 <-> ON. Also note that this value must be stepped up or down simultaneously on all servers; see the Manual for instructions.", - ErMasterHasPurgedRequiredGtids: "The slave is connecting using CHANGE MASTER TO MASTERAUTOPOSITION = 1, but the master has purged binary logs containing GTIDs that the slave requires.", - ErCantSetGtidNextWhenOwningGtid: "@@SESSION.GTIDNEXT cannot be changed by a client that owns a GTID. The client owns %s. Ownership is released on COMMIT or ROLLBACK.", - ErUnknownExplainFormat: "Unknown EXPLAIN format name: '%s'", - ErCantExecuteInReadOnlyTransaction: "Cannot execute statement in a READ ONLY transaction.", - ErTooLongTablePartitionComment: "Comment for table partition '%-.64s' is too long (max = %lu)", - ErSlaveConfiguration: "Slave is not configured or failed to initialize properly. You must at least set --server-id to enable either a master or a slave. Additional error messages can be found in the MySQL error log.", - ErInnodbFtLimit: "InnoDB presently supports one FULLTEXT index creation at a time", - ErInnodbNoFtTempTable: "Cannot create FULLTEXT index on temporary InnoDB table", - ErInnodbFtWrongDocidColumn: "Column '%-.192s' is of wrong type for an InnoDB FULLTEXT index", - ErInnodbFtWrongDocidIndex: "Index '%-.192s' is of wrong type for an InnoDB FULLTEXT index", - ErInnodbOnlineLogTooBig: "Creating index '%-.192s' required more than 'innodbOnlineAlterLogMaxSize' bytes of modification log. Please try again.", - ErUnknownAlterAlgorithm: "Unknown ALGORITHM '%s'", - ErUnknownAlterLock: "Unknown LOCK type '%s'", - ErMtsChangeMasterCantRunWithGaps: "CHANGE MASTER cannot be executed when the slave was stopped with an error or killed in MTS mode. Consider using RESET SLAVE or START SLAVE UNTIL.", - ErMtsRecoveryFailure: "Cannot recover after SLAVE errored out in parallel execution mode. Additional error messages can be found in the MySQL error log.", - ErMtsResetWorkers: "Cannot clean up worker info tables. Additional error messages can be found in the MySQL error log.", - ErColCountDoesntMatchCorruptedV2: "Column count of %s.%s is wrong. Expected %d, found %d. The table is probably corrupted", - ErSlaveSilentRetryTransaction: "Slave must silently retry current transaction", - ErDiscardFkChecksRunning: "There is a foreign key check running on table '%-.192s'. Cannot discard the table.", - ErTableSchemaMismatch: "Schema mismatch (%s)", - ErTableInSystemTablespace: "Table '%-.192s' in system tablespace", - ErIoReadError: "IO Read error: (%lu, %s) %s", - ErIoWriteError: "IO Write error: (%lu, %s) %s", - ErTablespaceMissing: "Tablespace is missing for table '%-.192s'", - ErTablespaceExists: "Tablespace for table '%-.192s' exists. Please DISCARD the tablespace before IMPORT.", - ErTablespaceDiscarded: "Tablespace has been discarded for table '%-.192s'", - ErInternalError: "Internal error: %s", - ErInnodbImportError: "ALTER TABLE '%-.192s' IMPORT TABLESPACE failed with error %lu : '%s'", - ErInnodbIndexCorrupt: "Index corrupt: %s", - ErInvalidYearColumnLength: "YEAR(%lu) column type is deprecated. Creating YEAR(4) column instead.", - ErNotValidPassword: "Your password does not satisfy the current policy requirements", - ErMustChangePassword: "You must SET PASSWORD before executing this statement", - ErFkNoIndexChild: "Failed to add the foreign key constaint. Missing index for constraint '%s' in the foreign table '%s'", - ErFkNoIndexParent: "Failed to add the foreign key constaint. Missing index for constraint '%s' in the referenced table '%s'", - ErFkFailAddSystem: "Failed to add the foreign key constraint '%s' to system tables", - ErFkCannotOpenParent: "Failed to open the referenced table '%s'", - ErFkIncorrectOption: "Failed to add the foreign key constraint on table '%s'. Incorrect options in FOREIGN KEY constraint '%s'", - ErFkDupName: "Duplicate foreign key constraint name '%s'", - ErPasswordFormat: "The password hash doesn't have the expected format. Check if the correct password algorithm is being used with the PASSWORD() function.", - ErFkColumnCannotDrop: "Cannot drop column '%-.192s': needed in a foreign key constraint '%-.192s'", - ErFkColumnCannotDropChild: "Cannot drop column '%-.192s': needed in a foreign key constraint '%-.192s' of table '%-.192s'", - ErFkColumnNotNull: "Column '%-.192s' cannot be NOT NULL: needed in a foreign key constraint '%-.192s' SET NULL", - ErDupIndex: "Duplicate index '%-.64s' defined on the table '%-.64s.%-.64s'. This is deprecated and will be disallowed in a future release.", - ErFkColumnCannotChange: "Cannot change column '%-.192s': used in a foreign key constraint '%-.192s'", - ErFkColumnCannotChangeChild: "Cannot change column '%-.192s': used in a foreign key constraint '%-.192s' of table '%-.192s'", - ErFkCannotDeleteParent: "Cannot delete rows from table which is parent in a foreign key constraint '%-.192s' of table '%-.192s'", - ErMalformedPacket: "Malformed communication packet.", - ErReadOnlyMode: "Running in read-only mode", - ErGtidNextTypeUndefinedGroup: "When @@SESSION.GTIDNEXT is set to a GTID, you must explicitly set it again after a COMMIT or ROLLBACK. If you see this error message in the slave SQL thread, it means that a table in the current transaction is transactional on the master and non-transactional on the slave. In a client connection, it means that you executed SET @@SESSION.GTIDNEXT before a transaction and forgot to set @@SESSION.GTIDNEXT to a different identifier or to 'AUTOMATIC' after COMMIT or ROLLBACK. Current @@SESSION.GTIDNEXT is '%s'.", - ErVariableNotSettableInSp: "The system variable %.200s cannot be set in stored procedures.", - ErCantSetGtidPurgedWhenGtidModeIsOff: "@@GLOBAL.GTIDPURGED can only be set when @@GLOBAL.GTIDMODE = ON.", - ErCantSetGtidPurgedWhenGtidExecutedIsNotEmpty: "@@GLOBAL.GTIDPURGED can only be set when @@GLOBAL.GTIDEXECUTED is empty.", - ErCantSetGtidPurgedWhenOwnedGtidsIsNotEmpty: "@@GLOBAL.GTIDPURGED can only be set when there are no ongoing transactions (not even in other clients).", - ErGtidPurgedWasChanged: "@@GLOBAL.GTIDPURGED was changed from '%s' to '%s'.", - ErGtidExecutedWasChanged: "@@GLOBAL.GTIDEXECUTED was changed from '%s' to '%s'.", - ErBinlogStmtModeAndNoReplTables: "Cannot execute statement: impossible to write to binary log since BINLOGFORMAT = STATEMENT, and both replicated and non replicated tables are written to.", - ErAlterOperationNotSupported: "%s is not supported for this operation. Try %s.", - ErAlterOperationNotSupportedReason: "%s is not supported. Reason: %s. Try %s.", - ErAlterOperationNotSupportedReasonCopy: "COPY algorithm requires a lock", - ErAlterOperationNotSupportedReasonPartition: "Partition specific operations do not yet support LOCK/ALGORITHM", - ErAlterOperationNotSupportedReasonFkRename: "Columns participating in a foreign key are renamed", - ErAlterOperationNotSupportedReasonColumnType: "Cannot change column type INPLACE", - ErAlterOperationNotSupportedReasonFkCheck: "Adding foreign keys needs foreignKeyChecks=OFF", - ErAlterOperationNotSupportedReasonIgnore: "Creating unique indexes with IGNORE requires COPY algorithm to remove duplicate rows", - ErAlterOperationNotSupportedReasonNopk: "Dropping a primary key is not allowed without also adding a new primary key", - ErAlterOperationNotSupportedReasonAutoinc: "Adding an auto-increment column requires a lock", - ErAlterOperationNotSupportedReasonHiddenFts: "Cannot replace hidden FTSDOCID with a user-visible one", - ErAlterOperationNotSupportedReasonChangeFts: "Cannot drop or rename FTSDOCID", - ErAlterOperationNotSupportedReasonFts: "Fulltext index creation requires a lock", - ErSQLSlaveSkipCounterNotSettableInGtidMode: "sqlSlaveSkipCounter can not be set when the server is running with @@GLOBAL.GTIDMODE = ON. Instead, for each transaction that you want to skip, generate an empty transaction with the same GTID as the transaction", - ErDupUnknownInIndex: "Duplicate entry for key '%-.192s'", - ErIdentCausesTooLongPath: "Long database name and identifier for object resulted in path length exceeding %d characters. Path: '%s'.", - ErAlterOperationNotSupportedReasonNotNull: "cannot silently convert NULL values, as required in this SQLMODE", - ErMustChangePasswordLogin: "Your password has expired. To log in you must change it using a client that supports expired passwords.", - ErRowInWrongPartition: "Found a row in wrong partition %s", + ErrHashchk: "hashchk", + ErrNisamchk: "isamchk", + ErrNo: "NO", + ErrYes: "YES", + ErrCantCreateFile: "Can't create file '%-.200s' (errno: %d - %s)", + ErrCantCreateTable: "Can't create table '%-.200s' (errno: %d)", + ErrCantCreateDb: "Can't create database '%-.192s' (errno: %d)", + ErrDbCreateExists: "Can't create database '%-.192s'; database exists", + ErrDbDropExists: "Can't drop database '%-.192s'; database doesn't exist", + ErrDbDropDelete: "Error dropping database (can't delete '%-.192s', errno: %d)", + ErrDbDropRmdir: "Error dropping database (can't rmdir '%-.192s', errno: %d)", + ErrCantDeleteFile: "Error on delete of '%-.192s' (errno: %d - %s)", + ErrCantFindSystemRec: "Can't read record in system table", + ErrCantGetStat: "Can't get status of '%-.200s' (errno: %d - %s)", + ErrCantGetWd: "Can't get working directory (errno: %d - %s)", + ErrCantLock: "Can't lock file (errno: %d - %s)", + ErrCantOpenFile: "Can't open file: '%-.200s' (errno: %d - %s)", + ErrFileNotFound: "Can't find file: '%-.200s' (errno: %d - %s)", + ErrCantReadDir: "Can't read dir of '%-.192s' (errno: %d - %s)", + ErrCantSetWd: "Can't change dir to '%-.192s' (errno: %d - %s)", + ErrCheckread: "Record has changed since last read in table '%-.192s'", + ErrDiskFull: "Disk full (%s); waiting for someone to free some space... (errno: %d - %s)", + ErrDupKey: "Can't write; duplicate key in table '%-.192s'", + ErrErrorOnClose: "Error on close of '%-.192s' (errno: %d - %s)", + ErrErrorOnRead: "Error reading file '%-.200s' (errno: %d - %s)", + ErrErrorOnRename: "Error on rename of '%-.210s' to '%-.210s' (errno: %d - %s)", + ErrErrorOnWrite: "Error writing file '%-.200s' (errno: %d - %s)", + ErrFileUsed: "'%-.192s' is locked against change", + ErrFilsortAbort: "Sort aborted", + ErrFormNotFound: "View '%-.192s' doesn't exist for '%-.192s'", + ErrGetErrno: "Got error %d from storage engine", + ErrIllegalHa: "Table storage engine for '%-.192s' doesn't have this option", + ErrKeyNotFound: "Can't find record in '%-.192s'", + ErrNotFormFile: "Incorrect information in file: '%-.200s'", + ErrNotKeyfile: "Incorrect key file for table '%-.200s'; try to repair it", + ErrOldKeyfile: "Old key file for table '%-.192s'; repair it!", + ErrOpenAsReadonly: "Table '%-.192s' is read only", + ErrOutofmemory: "Out of memory; restart server and try again (needed %d bytes)", + ErrOutOfSortmemory: "Out of sort memory, consider increasing server sort buffer size", + ErrUnexpectedEOF: "Unexpected EOF found when reading file '%-.192s' (errno: %d - %s)", + ErrConCount: "Too many connections", + ErrOutOfResources: "Out of memory; check if mysqld or some other process uses all available memory; if not, you may have to use 'ulimit' to allow mysqld to use more memory or you can add more swap space", + ErrBadHost: "Can't get hostname for your address", + ErrHandshake: "Bad handshake", + ErrDbaccessDenied: "Access denied for user '%-.48s'@'%-.64s' to database '%-.192s'", + ErrAccessDenied: "Access denied for user '%-.48s'@'%-.64s' (using password: %s)", + ErrNoDb: "No database selected", + ErrUnknownCom: "Unknown command", + ErrBadNull: "Column '%-.192s' cannot be null", + ErrBadDb: "Unknown database '%-.192s'", + ErrTableExists: "Table '%-.192s' already exists", + ErrBadTable: "Unknown table '%-.100s'", + ErrNonUniq: "Column '%-.192s' in %-.192s is ambiguous", + ErrServerShutdown: "Server shutdown in progress", + ErrBadField: "Unknown column '%-.192s' in '%-.192s'", + ErrWrongFieldWithGroup: "'%-.192s' isn't in GROUP BY", + ErrWrongGroupField: "Can't group on '%-.192s'", + ErrWrongSumSelect: "Statement has sum functions and columns in same statement", + ErrWrongValueCount: "Column count doesn't match value count", + ErrTooLongIdent: "Identifier name '%-.100s' is too long", + ErrDupFieldname: "Duplicate column name '%-.192s'", + ErrDupKeyname: "Duplicate key name '%-.192s'", + ErrDupEntry: "Duplicate entry '%-.192s' for key %d", + ErrWrongFieldSpec: "Incorrect column specifier for column '%-.192s'", + ErrParse: "%s near '%-.80s' at line %d", + ErrEmptyQuery: "Query was empty", + ErrNonuniqTable: "Not unique table/alias: '%-.192s'", + ErrInvalidDefault: "Invalid default value for '%-.192s'", + ErrMultiplePriKey: "Multiple primary key defined", + ErrTooManyKeys: "Too many keys specified; max %d keys allowed", + ErrTooManyKeyParts: "Too many key parts specified; max %d parts allowed", + ErrTooLongKey: "Specified key was too long; max key length is %d bytes", + ErrKeyColumnDoesNotExits: "Key column '%-.192s' doesn't exist in table", + ErrBlobUsedAsKey: "BLOB column '%-.192s' can't be used in key specification with the used table type", + ErrTooBigFieldlength: "Column length too big for column '%-.192s' (max = %lu); use BLOB or TEXT instead", + ErrWrongAutoKey: "Incorrect table definition; there can be only one auto column and it must be defined as a key", + ErrReady: "%s: ready for connections.\nVersion: '%s' socket: '%s' port: %d", + ErrNormalShutdown: "%s: Normal shutdown\n", + ErrGotSignal: "%s: Got signal %d. Aborting!\n", + ErrShutdownComplete: "%s: Shutdown complete\n", + ErrForcingClose: "%s: Forcing close of thread %ld user: '%-.48s'\n", + ErrIpsock: "Can't create IP socket", + ErrNoSuchIndex: "Table '%-.192s' has no index like the one used in CREATE INDEX; recreate the table", + ErrWrongFieldTerminators: "Field separator argument is not what is expected; check the manual", + ErrBlobsAndNoTerminated: "You can't use fixed rowlength with BLOBs; please use 'fields terminated by'", + ErrTextfileNotReadable: "The file '%-.128s' must be in the database directory or be readable by all", + ErrFileExists: "File '%-.200s' already exists", + ErrLoadInfo: "Records: %ld Deleted: %ld Skipped: %ld Warnings: %ld", + ErrAlterInfo: "Records: %ld Duplicates: %ld", + ErrWrongSubKey: "Incorrect prefix key; the used key part isn't a string, the used length is longer than the key part, or the storage engine doesn't support unique prefix keys", + ErrCantRemoveAllFields: "You can't delete all columns with ALTER TABLE; use DROP TABLE instead", + ErrCantDropFieldOrKey: "Can't DROP '%-.192s'; check that column/key exists", + ErrInsertInfo: "Records: %ld Duplicates: %ld Warnings: %ld", + ErrUpdateTableUsed: "You can't specify target table '%-.192s' for update in FROM clause", + ErrNoSuchThread: "Unknown thread id: %lu", + ErrKillDenied: "You are not owner of thread %lu", + ErrNoTablesUsed: "No tables used", + ErrTooBigSet: "Too many strings for column %-.192s and SET", + ErrNoUniqueLogfile: "Can't generate a unique log-filename %-.200s.(1-999)\n", + ErrTableNotLockedForWrite: "Table '%-.192s' was locked with a READ lock and can't be updated", + ErrTableNotLocked: "Table '%-.192s' was not locked with LOCK TABLES", + ErrBlobCantHaveDefault: "BLOB/TEXT column '%-.192s' can't have a default value", + ErrWrongDbName: "Incorrect database name '%-.100s'", + ErrWrongTableName: "Incorrect table name '%-.100s'", + ErrTooBigSelect: "The SELECT would examine more than MAXJOINSIZE rows; check your WHERE and use SET SQLBIGSELECTS=1 or SET MAXJOINSIZE=# if the SELECT is okay", + ErrUnknown: "Unknown error", + ErrUnknownProcedure: "Unknown procedure '%-.192s'", + ErrWrongParamcountToProcedure: "Incorrect parameter count to procedure '%-.192s'", + ErrWrongParametersToProcedure: "Incorrect parameters to procedure '%-.192s'", + ErrUnknownTable: "Unknown table '%-.192s' in %-.32s", + ErrFieldSpecifiedTwice: "Column '%-.192s' specified twice", + ErrInvalidGroupFuncUse: "Invalid use of group function", + ErrUnsupportedExtension: "Table '%-.192s' uses an extension that doesn't exist in this MySQL version", + ErrTableMustHaveColumns: "A table must have at least 1 column", + ErrRecordFileFull: "The table '%-.192s' is full", + ErrUnknownCharacterSet: "Unknown character set: '%-.64s'", + ErrTooManyTables: "Too many tables; MySQL can only use %d tables in a join", + ErrTooManyFields: "Too many columns", + ErrTooBigRowsize: "Row size too large. The maximum row size for the used table type, not counting BLOBs, is %ld. This includes storage overhead, check the manual. You have to change some columns to TEXT or BLOBs", + ErrStackOverrun: "Thread stack overrun: Used: %ld of a %ld stack. Use 'mysqld --threadStack=#' to specify a bigger stack if needed", + ErrWrongOuterJoin: "Cross dependency found in OUTER JOIN; examine your ON conditions", + ErrNullColumnInIndex: "Table handler doesn't support NULL in given index. Please change column '%-.192s' to be NOT NULL or use another handler", + ErrCantFindUdf: "Can't load function '%-.192s'", + ErrCantInitializeUdf: "Can't initialize function '%-.192s'; %-.80s", + ErrUdfNoPaths: "No paths allowed for shared library", + ErrUdfExists: "Function '%-.192s' already exists", + ErrCantOpenLibrary: "Can't open shared library '%-.192s' (errno: %d %-.128s)", + ErrCantFindDlEntry: "Can't find symbol '%-.128s' in library", + ErrFunctionNotDefined: "Function '%-.192s' is not defined", + ErrHostIsBlocked: "Host '%-.64s' is blocked because of many connection errors; unblock with 'mysqladmin flush-hosts'", + ErrHostNotPrivileged: "Host '%-.64s' is not allowed to connect to this MySQL server", + ErrPasswordAnonymousUser: "You are using MySQL as an anonymous user and anonymous users are not allowed to change passwords", + ErrPasswordNotAllowed: "You must have privileges to update tables in the mysql database to be able to change passwords for others", + ErrPasswordNoMatch: "Can't find any matching row in the user table", + ErrUpdateInfo: "Rows matched: %ld Changed: %ld Warnings: %ld", + ErrCantCreateThread: "Can't create a new thread (errno %d); if you are not out of available memory, you can consult the manual for a possible OS-dependent bug", + ErrWrongValueCountOnRow: "Column count doesn't match value count at row %ld", + ErrCantReopenTable: "Can't reopen table: '%-.192s'", + ErrInvalidUseOfNull: "Invalid use of NULL value", + ErrRegexp: "Got error '%-.64s' from regexp", + ErrMixOfGroupFuncAndFields: "Mixing of GROUP columns (MIN(),MAX(),COUNT(),...) with no GROUP columns is illegal if there is no GROUP BY clause", + ErrNonexistingGrant: "There is no such grant defined for user '%-.48s' on host '%-.64s'", + ErrTableaccessDenied: "%-.128s command denied to user '%-.48s'@'%-.64s' for table '%-.64s'", + ErrColumnaccessDenied: "%-.16s command denied to user '%-.48s'@'%-.64s' for column '%-.192s' in table '%-.192s'", + ErrIllegalGrantForTable: "Illegal GRANT/REVOKE command; please consult the manual to see which privileges can be used", + ErrGrantWrongHostOrUser: "The host or user argument to GRANT is too long", + ErrNoSuchTable: "Table '%-.192s.%-.192s' doesn't exist", + ErrNonexistingTableGrant: "There is no such grant defined for user '%-.48s' on host '%-.64s' on table '%-.192s'", + ErrNotAllowedCommand: "The used command is not allowed with this MySQL version", + ErrSyntax: "You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use", + ErrDelayedCantChangeLock: "Delayed insert thread couldn't get requested lock for table %-.192s", + ErrTooManyDelayedThreads: "Too many delayed threads in use", + ErrAbortingConnection: "Aborted connection %ld to db: '%-.192s' user: '%-.48s' (%-.64s)", + ErrNetPacketTooLarge: "Got a packet bigger than 'maxAllowedPacket' bytes", + ErrNetReadErrorFromPipe: "Got a read error from the connection pipe", + ErrNetFcntl: "Got an error from fcntl()", + ErrNetPacketsOutOfOrder: "Got packets out of order", + ErrNetUncompress: "Couldn't uncompress communication packet", + ErrNetRead: "Got an error reading communication packets", + ErrNetReadInterrupted: "Got timeout reading communication packets", + ErrNetErrorOnWrite: "Got an error writing communication packets", + ErrNetWriteInterrupted: "Got timeout writing communication packets", + ErrTooLongString: "Result string is longer than 'maxAllowedPacket' bytes", + ErrTableCantHandleBlob: "The used table type doesn't support BLOB/TEXT columns", + ErrTableCantHandleAutoIncrement: "The used table type doesn't support AUTOINCREMENT columns", + ErrDelayedInsertTableLocked: "INSERT DELAYED can't be used with table '%-.192s' because it is locked with LOCK TABLES", + ErrWrongColumnName: "Incorrect column name '%-.100s'", + ErrWrongKeyColumn: "The used storage engine can't index column '%-.192s'", + ErrWrongMrgTable: "Unable to open underlying table which is differently defined or of non-MyISAM type or doesn't exist", + ErrDupUnique: "Can't write, because of unique constraint, to table '%-.192s'", + ErrBlobKeyWithoutLength: "BLOB/TEXT column '%-.192s' used in key specification without a key length", + ErrPrimaryCantHaveNull: "All parts of a PRIMARY KEY must be NOT NULL; if you need NULL in a key, use UNIQUE instead", + ErrTooManyRows: "Result consisted of more than one row", + ErrRequiresPrimaryKey: "This table type requires a primary key", + ErrNoRaidCompiled: "This version of MySQL is not compiled with RAID support", + ErrUpdateWithoutKeyInSafeMode: "You are using safe update mode and you tried to update a table without a WHERE that uses a KEY column", + ErrKeyDoesNotExits: "Key '%-.192s' doesn't exist in table '%-.192s'", + ErrCheckNoSuchTable: "Can't open table", + ErrCheckNotImplemented: "The storage engine for the table doesn't support %s", + ErrCantDoThisDuringAnTransaction: "You are not allowed to execute this command in a transaction", + ErrErrorDuringCommit: "Got error %d during COMMIT", + ErrErrorDuringRollback: "Got error %d during ROLLBACK", + ErrErrorDuringFlushLogs: "Got error %d during FLUSHLOGS", + ErrErrorDuringCheckpoint: "Got error %d during CHECKPOINT", + ErrNewAbortingConnection: "Aborted connection %ld to db: '%-.192s' user: '%-.48s' host: '%-.64s' (%-.64s)", + ErrDumpNotImplemented: "The storage engine for the table does not support binary table dump", + ErrFlushMasterBinlogClosed: "Binlog closed, cannot RESET MASTER", + ErrIndexRebuild: "Failed rebuilding the index of dumped table '%-.192s'", + ErrMaster: "Error from master: '%-.64s'", + ErrMasterNetRead: "Net error reading from master", + ErrMasterNetWrite: "Net error writing to master", + ErrFtMatchingKeyNotFound: "Can't find FULLTEXT index matching the column list", + ErrLockOrActiveTransaction: "Can't execute the given command because you have active locked tables or an active transaction", + ErrUnknownSystemVariable: "Unknown system variable '%-.64s'", + ErrCrashedOnUsage: "Table '%-.192s' is marked as crashed and should be repaired", + ErrCrashedOnRepair: "Table '%-.192s' is marked as crashed and last (automatic?) repair failed", + ErrWarningNotCompleteRollback: "Some non-transactional changed tables couldn't be rolled back", + ErrTransCacheFull: "Multi-statement transaction required more than 'maxBinlogCacheSize' bytes of storage; increase this mysqld variable and try again", + ErrSlaveMustStop: "This operation cannot be performed with a running slave; run STOP SLAVE first", + ErrSlaveNotRunning: "This operation requires a running slave; configure slave and do START SLAVE", + ErrBadSlave: "The server is not configured as slave; fix in config file or with CHANGE MASTER TO", + ErrMasterInfo: "Could not initialize master info structure; more error messages can be found in the MySQL error log", + ErrSlaveThread: "Could not create slave thread; check system resources", + ErrTooManyUserConnections: "User %-.64s already has more than 'maxUserConnections' active connections", + ErrSetConstantsOnly: "You may only use constant expressions with SET", + ErrLockWaitTimeout: "Lock wait timeout exceeded; try restarting transaction", + ErrLockTableFull: "The total number of locks exceeds the lock table size", + ErrReadOnlyTransaction: "Update locks cannot be acquired during a READ UNCOMMITTED transaction", + ErrDropDbWithReadLock: "DROP DATABASE not allowed while thread is holding global read lock", + ErrCreateDbWithReadLock: "CREATE DATABASE not allowed while thread is holding global read lock", + ErrWrongArguments: "Incorrect arguments to %s", + ErrNoPermissionToCreateUser: "'%-.48s'@'%-.64s' is not allowed to create new users", + ErrUnionTablesInDifferentDir: "Incorrect table definition; all MERGE tables must be in the same database", + ErrLockDeadlock: "Deadlock found when trying to get lock; try restarting transaction", + ErrTableCantHandleFt: "The used table type doesn't support FULLTEXT indexes", + ErrCannotAddForeign: "Cannot add foreign key constraint", + ErrNoReferencedRow: "Cannot add or update a child row: a foreign key constraint fails", + ErrRowIsReferenced: "Cannot delete or update a parent row: a foreign key constraint fails", + ErrConnectToMaster: "Error connecting to master: %-.128s", + ErrQueryOnMaster: "Error running query on master: %-.128s", + ErrErrorWhenExecutingCommand: "Error when executing command %s: %-.128s", + ErrWrongUsage: "Incorrect usage of %s and %s", + ErrWrongNumberOfColumnsInSelect: "The used SELECT statements have a different number of columns", + ErrCantUpdateWithReadlock: "Can't execute the query because you have a conflicting read lock", + ErrMixingNotAllowed: "Mixing of transactional and non-transactional tables is disabled", + ErrDupArgument: "Option '%s' used twice in statement", + ErrUserLimitReached: "User '%-.64s' has exceeded the '%s' resource (current value: %ld)", + ErrSpecificAccessDenied: "Access denied; you need (at least one of) the %-.128s privilege(s) for this operation", + ErrLocalVariable: "Variable '%-.64s' is a SESSION variable and can't be used with SET GLOBAL", + ErrGlobalVariable: "Variable '%-.64s' is a GLOBAL variable and should be set with SET GLOBAL", + ErrNoDefault: "Variable '%-.64s' doesn't have a default value", + ErrWrongValueForVar: "Variable '%-.64s' can't be set to the value of '%-.200s'", + ErrWrongTypeForVar: "Incorrect argument type to variable '%-.64s'", + ErrVarCantBeRead: "Variable '%-.64s' can only be set, not read", + ErrCantUseOptionHere: "Incorrect usage/placement of '%s'", + ErrNotSupportedYet: "This version of MySQL doesn't yet support '%s'", + ErrMasterFatalErrorReadingBinlog: "Got fatal error %d from master when reading data from binary log: '%-.320s'", + ErrSlaveIgnoredTable: "Slave SQL thread ignored the query because of replicate-*-table rules", + ErrIncorrectGlobalLocalVar: "Variable '%-.192s' is a %s variable", + ErrWrongFkDef: "Incorrect foreign key definition for '%-.192s': %s", + ErrKeyRefDoNotMatchTableRef: "Key reference and table reference don't match", + ErrOperandColumns: "Operand should contain %d column(s)", + ErrSubqueryNo1Row: "Subquery returns more than 1 row", + ErrUnknownStmtHandler: "Unknown prepared statement handler (%.*s) given to %s", + ErrCorruptHelpDb: "Help database is corrupt or does not exist", + ErrCyclicReference: "Cyclic reference on subqueries", + ErrAutoConvert: "Converting column '%s' from %s to %s", + ErrIllegalReference: "Reference '%-.64s' not supported (%s)", + ErrDerivedMustHaveAlias: "Every derived table must have its own alias", + ErrSelectReduced: "Select %u was reduced during optimization", + ErrTablenameNotAllowedHere: "Table '%-.192s' from one of the SELECTs cannot be used in %-.32s", + ErrNotSupportedAuthMode: "Client does not support authentication protocol requested by server; consider upgrading MySQL client", + ErrSpatialCantHaveNull: "All parts of a SPATIAL index must be NOT NULL", + ErrCollationCharsetMismatch: "COLLATION '%s' is not valid for CHARACTER SET '%s'", + ErrSlaveWasRunning: "Slave is already running", + ErrSlaveWasNotRunning: "Slave already has been stopped", + ErrTooBigForUncompress: "Uncompressed data size too large; the maximum size is %d (probably, length of uncompressed data was corrupted)", + ErrZlibZMem: "ZLIB: Not enough memory", + ErrZlibZBuf: "ZLIB: Not enough room in the output buffer (probably, length of uncompressed data was corrupted)", + ErrZlibZData: "ZLIB: Input data corrupted", + ErrCutValueGroupConcat: "Row %u was cut by GROUPCONCAT()", + ErrWarnTooFewRecords: "Row %ld doesn't contain data for all columns", + ErrWarnTooManyRecords: "Row %ld was truncated; it contained more data than there were input columns", + ErrWarnNullToNotnull: "Column set to default value; NULL supplied to NOT NULL column '%s' at row %ld", + ErrWarnDataOutOfRange: "Out of range value for column '%s' at row %ld", + WarnDataTruncated: "Data truncated for column '%s' at row %ld", + ErrWarnUsingOtherHandler: "Using storage engine %s for table '%s'", + ErrCantAggregate2collations: "Illegal mix of collations (%s,%s) and (%s,%s) for operation '%s'", + ErrDropUser: "Cannot drop one or more of the requested users", + ErrRevokeGrants: "Can't revoke all privileges for one or more of the requested users", + ErrCantAggregate3collations: "Illegal mix of collations (%s,%s), (%s,%s), (%s,%s) for operation '%s'", + ErrCantAggregateNcollations: "Illegal mix of collations for operation '%s'", + ErrVariableIsNotStruct: "Variable '%-.64s' is not a variable component (can't be used as XXXX.variableName)", + ErrUnknownCollation: "Unknown collation: '%-.64s'", + ErrSlaveIgnoredSslParams: "SSL parameters in CHANGE MASTER are ignored because this MySQL slave was compiled without SSL support; they can be used later if MySQL slave with SSL is started", + ErrServerIsInSecureAuthMode: "Server is running in --secure-auth mode, but '%s'@'%s' has a password in the old format; please change the password to the new format", + ErrWarnFieldResolved: "Field or reference '%-.192s%s%-.192s%s%-.192s' of SELECT #%d was resolved in SELECT #%d", + ErrBadSlaveUntilCond: "Incorrect parameter or combination of parameters for START SLAVE UNTIL", + ErrMissingSkipSlave: "It is recommended to use --skip-slave-start when doing step-by-step replication with START SLAVE UNTIL; otherwise, you will get problems if you get an unexpected slave's mysqld restart", + ErrUntilCondIgnored: "SQL thread is not to be started so UNTIL options are ignored", + ErrWrongNameForIndex: "Incorrect index name '%-.100s'", + ErrWrongNameForCatalog: "Incorrect catalog name '%-.100s'", + ErrWarnQcResize: "Query cache failed to set size %lu; new query cache size is %lu", + ErrBadFtColumn: "Column '%-.192s' cannot be part of FULLTEXT index", + ErrUnknownKeyCache: "Unknown key cache '%-.100s'", + ErrWarnHostnameWontWork: "MySQL is started in --skip-name-resolve mode; you must restart it without this switch for this grant to work", + ErrUnknownStorageEngine: "Unknown storage engine '%s'", + ErrWarnDeprecatedSyntax: "'%s' is deprecated and will be removed in a future release. Please use %s instead", + ErrNonUpdatableTable: "The target table %-.100s of the %s is not updatable", + ErrFeatureDisabled: "The '%s' feature is disabled; you need MySQL built with '%s' to have it working", + ErrOptionPreventsStatement: "The MySQL server is running with the %s option so it cannot execute this statement", + ErrDuplicatedValueInType: "Column '%-.100s' has duplicated value '%-.64s' in %s", + ErrTruncatedWrongValue: "Truncated incorrect %-.32s value: '%-.128s'", + ErrTooMuchAutoTimestampCols: "Incorrect table definition; there can be only one TIMESTAMP column with CURRENTTIMESTAMP in DEFAULT or ON UPDATE clause", + ErrInvalidOnUpdate: "Invalid ON UPDATE clause for '%-.192s' column", + ErrUnsupportedPs: "This command is not supported in the prepared statement protocol yet", + ErrGetErrmsg: "Got error %d '%-.100s' from %s", + ErrGetTemporaryErrmsg: "Got temporary error %d '%-.100s' from %s", + ErrUnknownTimeZone: "Unknown or incorrect time zone: '%-.64s'", + ErrWarnInvalidTimestamp: "Invalid TIMESTAMP value in column '%s' at row %ld", + ErrInvalidCharacterString: "Invalid %s character string: '%.64s'", + ErrWarnAllowedPacketOverflowed: "Result of %s() was larger than maxAllowedPacket (%ld) - truncated", + ErrConflictingDeclarations: "Conflicting declarations: '%s%s' and '%s%s'", + ErrSpNoRecursiveCreate: "Can't create a %s from within another stored routine", + ErrSpAlreadyExists: "%s %s already exists", + ErrSpDoesNotExist: "%s %s does not exist", + ErrSpDropFailed: "Failed to DROP %s %s", + ErrSpStoreFailed: "Failed to CREATE %s %s", + ErrSpLilabelMismatch: "%s with no matching label: %s", + ErrSpLabelRedefine: "Redefining label %s", + ErrSpLabelMismatch: "End-label %s without match", + ErrSpUninitVar: "Referring to uninitialized variable %s", + ErrSpBadselect: "PROCEDURE %s can't return a result set in the given context", + ErrSpBadreturn: "RETURN is only allowed in a FUNCTION", + ErrSpBadstatement: "%s is not allowed in stored procedures", + ErrUpdateLogDeprecatedIgnored: "The update log is deprecated and replaced by the binary log; SET SQLLOGUPDATE has been ignored.", + ErrUpdateLogDeprecatedTranslated: "The update log is deprecated and replaced by the binary log; SET SQLLOGUPDATE has been translated to SET SQLLOGBIN.", + ErrQueryInterrupted: "Query execution was interrupted", + ErrSpWrongNoOfArgs: "Incorrect number of arguments for %s %s; expected %u, got %u", + ErrSpCondMismatch: "Undefined CONDITION: %s", + ErrSpNoreturn: "No RETURN found in FUNCTION %s", + ErrSpNoreturnend: "FUNCTION %s ended without RETURN", + ErrSpBadCursorQuery: "Cursor statement must be a SELECT", + ErrSpBadCursorSelect: "Cursor SELECT must not have INTO", + ErrSpCursorMismatch: "Undefined CURSOR: %s", + ErrSpCursorAlreadyOpen: "Cursor is already open", + ErrSpCursorNotOpen: "Cursor is not open", + ErrSpUndeclaredVar: "Undeclared variable: %s", + ErrSpWrongNoOfFetchArgs: "Incorrect number of FETCH variables", + ErrSpFetchNoData: "No data - zero rows fetched, selected, or processed", + ErrSpDupParam: "Duplicate parameter: %s", + ErrSpDupVar: "Duplicate variable: %s", + ErrSpDupCond: "Duplicate condition: %s", + ErrSpDupCurs: "Duplicate cursor: %s", + ErrSpCantAlter: "Failed to ALTER %s %s", + ErrSpSubselectNyi: "Subquery value not supported", + ErrStmtNotAllowedInSfOrTrg: "%s is not allowed in stored function or trigger", + ErrSpVarcondAfterCurshndlr: "Variable or condition declaration after cursor or handler declaration", + ErrSpCursorAfterHandler: "Cursor declaration after handler declaration", + ErrSpCaseNotFound: "Case not found for CASE statement", + ErrFparserTooBigFile: "Configuration file '%-.192s' is too big", + ErrFparserBadHeader: "Malformed file type header in file '%-.192s'", + ErrFparserEOFInComment: "Unexpected end of file while parsing comment '%-.200s'", + ErrFparserErrorInParameter: "Error while parsing parameter '%-.192s' (line: '%-.192s')", + ErrFparserEOFInUnknownParameter: "Unexpected end of file while skipping unknown parameter '%-.192s'", + ErrViewNoExplain: "EXPLAIN/SHOW can not be issued; lacking privileges for underlying table", + ErrFrmUnknownType: "File '%-.192s' has unknown type '%-.64s' in its header", + ErrWrongObject: "'%-.192s.%-.192s' is not %s", + ErrNonupdateableColumn: "Column '%-.192s' is not updatable", + ErrViewSelectDerived: "View's SELECT contains a subquery in the FROM clause", + ErrViewSelectClause: "View's SELECT contains a '%s' clause", + ErrViewSelectVariable: "View's SELECT contains a variable or parameter", + ErrViewSelectTmptable: "View's SELECT refers to a temporary table '%-.192s'", + ErrViewWrongList: "View's SELECT and view's field list have different column counts", + ErrWarnViewMerge: "View merge algorithm can't be used here for now (assumed undefined algorithm)", + ErrWarnViewWithoutKey: "View being updated does not have complete key of underlying table in it", + ErrViewInvalid: "View '%-.192s.%-.192s' references invalid table(s) or column(s) or function(s) or definer/invoker of view lack rights to use them", + ErrSpNoDropSp: "Can't drop or alter a %s from within another stored routine", + ErrSpGotoInHndlr: "GOTO is not allowed in a stored procedure handler", + ErrTrgAlreadyExists: "Trigger already exists", + ErrTrgDoesNotExist: "Trigger does not exist", + ErrTrgOnViewOrTempTable: "Trigger's '%-.192s' is view or temporary table", + ErrTrgCantChangeRow: "Updating of %s row is not allowed in %strigger", + ErrTrgNoSuchRowInTrg: "There is no %s row in %s trigger", + ErrNoDefaultForField: "Field '%-.192s' doesn't have a default value", + ErrDivisionByZero: "Division by 0", + ErrTruncatedWrongValueForField: "Incorrect %-.32s value: '%-.128s' for column '%.192s' at row %ld", + ErrIllegalValueForType: "Illegal %s '%-.192s' value found during parsing", + ErrViewNonupdCheck: "CHECK OPTION on non-updatable view '%-.192s.%-.192s'", + ErrViewCheckFailed: "CHECK OPTION failed '%-.192s.%-.192s'", + ErrProcaccessDenied: "%-.16s command denied to user '%-.48s'@'%-.64s' for routine '%-.192s'", + ErrRelayLogFail: "Failed purging old relay logs: %s", + ErrPasswdLength: "Password hash should be a %d-digit hexadecimal number", + ErrUnknownTargetBinlog: "Target log not found in binlog index", + ErrIoErrLogIndexRead: "I/O error reading log index file", + ErrBinlogPurgeProhibited: "Server configuration does not permit binlog purge", + ErrFseekFail: "Failed on fseek()", + ErrBinlogPurgeFatalErr: "Fatal error during log purge", + ErrLogInUse: "A purgeable log is in use, will not purge", + ErrLogPurgeUnknownErr: "Unknown error during log purge", + ErrRelayLogInit: "Failed initializing relay log position: %s", + ErrNoBinaryLogging: "You are not using binary logging", + ErrReservedSyntax: "The '%-.64s' syntax is reserved for purposes internal to the MySQL server", + ErrWsasFailed: "WSAStartup Failed", + ErrDiffGroupsProc: "Can't handle procedures with different groups yet", + ErrNoGroupForProc: "Select must have a group with this procedure", + ErrOrderWithProc: "Can't use ORDER clause with this procedure", + ErrLoggingProhibitChangingOf: "Binary logging and replication forbid changing the global server %s", + ErrNoFileMapping: "Can't map file: %-.200s, errno: %d", + ErrWrongMagic: "Wrong magic in %-.64s", + ErrPsManyParam: "Prepared statement contains too many placeholders", + ErrKeyPart0: "Key part '%-.192s' length cannot be 0", + ErrViewChecksum: "View text checksum failed", + ErrViewMultiupdate: "Can not modify more than one base table through a join view '%-.192s.%-.192s'", + ErrViewNoInsertFieldList: "Can not insert into join view '%-.192s.%-.192s' without fields list", + ErrViewDeleteMergeView: "Can not delete from join view '%-.192s.%-.192s'", + ErrCannotUser: "Operation %s failed for %.256s", + ErrXaerNota: "XAERNOTA: Unknown XID", + ErrXaerInval: "XAERINVAL: Invalid arguments (or unsupported command)", + ErrXaerRmfail: "XAERRMFAIL: The command cannot be executed when global transaction is in the %.64s state", + ErrXaerOutside: "XAEROUTSIDE: Some work is done outside global transaction", + ErrXaerRmerr: "XAERRMERR: Fatal error occurred in the transaction branch - check your data for consistency", + ErrXaRbrollback: "XARBROLLBACK: Transaction branch was rolled back", + ErrNonexistingProcGrant: "There is no such grant defined for user '%-.48s' on host '%-.64s' on routine '%-.192s'", + ErrProcAutoGrantFail: "Failed to grant EXECUTE and ALTER ROUTINE privileges", + ErrProcAutoRevokeFail: "Failed to revoke all privileges to dropped routine", + ErrDataTooLong: "Data too long for column '%s' at row %ld", + ErrSpBadSQLstate: "Bad SQLSTATE: '%s'", + ErrStartup: "%s: ready for connections.\nVersion: '%s' socket: '%s' port: %d %s", + ErrLoadFromFixedSizeRowsToVar: "Can't load value from file with fixed size rows to variable", + ErrCantCreateUserWithGrant: "You are not allowed to create a user with GRANT", + ErrWrongValueForType: "Incorrect %-.32s value: '%-.128s' for function %-.32s", + ErrTableDefChanged: "Table definition has changed, please retry transaction", + ErrSpDupHandler: "Duplicate handler declared in the same block", + ErrSpNotVarArg: "OUT or INOUT argument %d for routine %s is not a variable or NEW pseudo-variable in BEFORE trigger", + ErrSpNoRetset: "Not allowed to return a result set from a %s", + ErrCantCreateGeometryObject: "Cannot get geometry object from data you send to the GEOMETRY field", + ErrFailedRoutineBreakBinlog: "A routine failed and has neither NO SQL nor READS SQL DATA in its declaration and binary logging is enabled; if non-transactional tables were updated, the binary log will miss their changes", + ErrBinlogUnsafeRoutine: "This function has none of DETERMINISTIC, NO SQL, or READS SQL DATA in its declaration and binary logging is enabled (you *might* want to use the less safe logBinTrustFunctionCreators variable)", + ErrBinlogCreateRoutineNeedSuper: "You do not have the SUPER privilege and binary logging is enabled (you *might* want to use the less safe logBinTrustFunctionCreators variable)", + ErrExecStmtWithOpenCursor: "You can't execute a prepared statement which has an open cursor associated with it. Reset the statement to re-execute it.", + ErrStmtHasNoOpenCursor: "The statement (%lu) has no open cursor.", + ErrCommitNotAllowedInSfOrTrg: "Explicit or implicit commit is not allowed in stored function or trigger.", + ErrNoDefaultForViewField: "Field of view '%-.192s.%-.192s' underlying table doesn't have a default value", + ErrSpNoRecursion: "Recursive stored functions and triggers are not allowed.", + ErrTooBigScale: "Too big scale %d specified for column '%-.192s'. Maximum is %lu.", + ErrTooBigPrecision: "Too big precision %d specified for column '%-.192s'. Maximum is %lu.", + ErrMBiggerThanD: "For float(M,D), double(M,D) or decimal(M,D), M must be >= D (column '%-.192s').", + ErrWrongLockOfSystemTable: "You can't combine write-locking of system tables with other tables or lock types", + ErrConnectToForeignDataSource: "Unable to connect to foreign data source: %.64s", + ErrQueryOnForeignDataSource: "There was a problem processing the query on the foreign data source. Data source : %-.64s", + ErrForeignDataSourceDoesntExist: "The foreign data source you are trying to reference does not exist. Data source : %-.64s", + ErrForeignDataStringInvalidCantCreate: "Can't create federated table. The data source connection string '%-.64s' is not in the correct format", + ErrForeignDataStringInvalid: "The data source connection string '%-.64s' is not in the correct format", + ErrCantCreateFederatedTable: "Can't create federated table. Foreign data src : %-.64s", + ErrTrgInWrongSchema: "Trigger in wrong schema", + ErrStackOverrunNeedMore: "Thread stack overrun: %ld bytes used of a %ld byte stack, and %ld bytes needed. Use 'mysqld --threadStack=#' to specify a bigger stack.", + ErrTooLongBody: "Routine body for '%-.100s' is too long", + ErrWarnCantDropDefaultKeycache: "Cannot drop default keycache", + ErrTooBigDisplaywidth: "Display width out of range for column '%-.192s' (max = %lu)", + ErrXaerDupid: "XAERDUPID: The XID already exists", + ErrDatetimeFunctionOverflow: "Datetime function: %-.32s field overflow", + ErrCantUpdateUsedTableInSfOrTrg: "Can't update table '%-.192s' in stored function/trigger because it is already used by statement which invoked this stored function/trigger.", + ErrViewPreventUpdate: "The definition of table '%-.192s' prevents operation %.192s on table '%-.192s'.", + ErrPsNoRecursion: "The prepared statement contains a stored routine call that refers to that same statement. It's not allowed to execute a prepared statement in such a recursive manner", + ErrSpCantSetAutocommit: "Not allowed to set autocommit from a stored function or trigger", + ErrMalformedDefiner: "Definer is not fully qualified", + ErrViewFrmNoUser: "View '%-.192s'.'%-.192s' has no definer information (old table format). Current user is used as definer. Please recreate the view!", + ErrViewOtherUser: "You need the SUPER privilege for creation view with '%-.192s'@'%-.192s' definer", + ErrNoSuchUser: "The user specified as a definer ('%-.64s'@'%-.64s') does not exist", + ErrForbidSchemaChange: "Changing schema from '%-.192s' to '%-.192s' is not allowed.", + ErrRowIsReferenced2: "Cannot delete or update a parent row: a foreign key constraint fails (%.192s)", + ErrNoReferencedRow2: "Cannot add or update a child row: a foreign key constraint fails (%.192s)", + ErrSpBadVarShadow: "Variable '%-.64s' must be quoted with `...`, or renamed", + ErrTrgNoDefiner: "No definer attribute for trigger '%-.192s'.'%-.192s'. The trigger will be activated under the authorization of the invoker, which may have insufficient privileges. Please recreate the trigger.", + ErrOldFileFormat: "'%-.192s' has an old format, you should re-create the '%s' object(s)", + ErrSpRecursionLimit: "Recursive limit %d (as set by the maxSpRecursionDepth variable) was exceeded for routine %.192s", + ErrSpProcTableCorrupt: "Failed to load routine %-.192s. The table mysql.proc is missing, corrupt, or contains bad data (internal code %d)", + ErrSpWrongName: "Incorrect routine name '%-.192s'", + ErrTableNeedsUpgrade: "Table upgrade required. Please do \"REPAIR TABLE `%-.32s`\"", + ErrSpNoAggregate: "AGGREGATE is not supported for stored functions", + ErrMaxPreparedStmtCountReached: "Can't create more than maxPreparedStmtCount statements (current value: %lu)", + ErrViewRecursive: "`%-.192s`.`%-.192s` contains view recursion", + ErrNonGroupingFieldUsed: "Non-grouping field '%-.192s' is used in %-.64s clause", + ErrTableCantHandleSpkeys: "The used table type doesn't support SPATIAL indexes", + ErrNoTriggersOnSystemSchema: "Triggers can not be created on system tables", + ErrRemovedSpaces: "Leading spaces are removed from name '%s'", + ErrAutoincReadFailed: "Failed to read auto-increment value from storage engine", + ErrUsername: "user name", + ErrHostname: "host name", + ErrWrongStringLength: "String '%-.70s' is too long for %s (should be no longer than %d)", + ErrNonInsertableTable: "The target table %-.100s of the %s is not insertable-into", + ErrAdminWrongMrgTable: "Table '%-.64s' is differently defined or of non-MyISAM type or doesn't exist", + ErrTooHighLevelOfNestingForSelect: "Too high level of nesting for select", + ErrNameBecomesEmpty: "Name '%-.64s' has become ''", + ErrAmbiguousFieldTerm: "First character of the FIELDS TERMINATED string is ambiguous; please use non-optional and non-empty FIELDS ENCLOSED BY", + ErrForeignServerExists: "The foreign server, %s, you are trying to create already exists.", + ErrForeignServerDoesntExist: "The foreign server name you are trying to reference does not exist. Data source : %-.64s", + ErrIllegalHaCreateOption: "Table storage engine '%-.64s' does not support the create option '%.64s'", + ErrPartitionRequiresValues: "Syntax : %-.64s PARTITIONING requires definition of VALUES %-.64s for each partition", + ErrPartitionWrongValues: "Only %-.64s PARTITIONING can use VALUES %-.64s in partition definition", + ErrPartitionMaxvalue: "MAXVALUE can only be used in last partition definition", + ErrPartitionSubpartition: "Subpartitions can only be hash partitions and by key", + ErrPartitionSubpartMix: "Must define subpartitions on all partitions if on one partition", + ErrPartitionWrongNoPart: "Wrong number of partitions defined, mismatch with previous setting", + ErrPartitionWrongNoSubpart: "Wrong number of subpartitions defined, mismatch with previous setting", + ErrWrongExprInPartitionFunc: "Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed", + ErrNoConstExprInRangeOrList: "Expression in RANGE/LIST VALUES must be constant", + ErrFieldNotFoundPart: "Field in list of fields for partition function not found in table", + ErrListOfFieldsOnlyInHash: "List of fields is only allowed in KEY partitions", + ErrInconsistentPartitionInfo: "The partition info in the frm file is not consistent with what can be written into the frm file", + ErrPartitionFuncNotAllowed: "The %-.192s function returns the wrong type", + ErrPartitionsMustBeDefined: "For %-.64s partitions each partition must be defined", + ErrRangeNotIncreasing: "VALUES LESS THAN value must be strictly increasing for each partition", + ErrInconsistentTypeOfFunctions: "VALUES value must be of same type as partition function", + ErrMultipleDefConstInListPart: "Multiple definition of same constant in list partitioning", + ErrPartitionEntry: "Partitioning can not be used stand-alone in query", + ErrMixHandler: "The mix of handlers in the partitions is not allowed in this version of MySQL", + ErrPartitionNotDefined: "For the partitioned engine it is necessary to define all %-.64s", + ErrTooManyPartitions: "Too many partitions (including subpartitions) were defined", + ErrSubpartition: "It is only possible to mix RANGE/LIST partitioning with HASH/KEY partitioning for subpartitioning", + ErrCantCreateHandlerFile: "Failed to create specific handler file", + ErrBlobFieldInPartFunc: "A BLOB field is not allowed in partition function", + ErrUniqueKeyNeedAllFieldsInPf: "A %-.192s must include all columns in the table's partitioning function", + ErrNoParts: "Number of %-.64s = 0 is not an allowed value", + ErrPartitionMgmtOnNonpartitioned: "Partition management on a not partitioned table is not possible", + ErrForeignKeyOnPartitioned: "Foreign key clause is not yet supported in conjunction with partitioning", + ErrDropPartitionNonExistent: "Error in list of partitions to %-.64s", + ErrDropLastPartition: "Cannot remove all partitions, use DROP TABLE instead", + ErrCoalesceOnlyOnHashPartition: "COALESCE PARTITION can only be used on HASH/KEY partitions", + ErrReorgHashOnlyOnSameNo: "REORGANIZE PARTITION can only be used to reorganize partitions not to change their numbers", + ErrReorgNoParam: "REORGANIZE PARTITION without parameters can only be used on auto-partitioned tables using HASH PARTITIONs", + ErrOnlyOnRangeListPartition: "%-.64s PARTITION can only be used on RANGE/LIST partitions", + ErrAddPartitionSubpart: "Trying to Add partition(s) with wrong number of subpartitions", + ErrAddPartitionNoNewPartition: "At least one partition must be added", + ErrCoalescePartitionNoPartition: "At least one partition must be coalesced", + ErrReorgPartitionNotExist: "More partitions to reorganize than there are partitions", + ErrSameNamePartition: "Duplicate partition name %-.192s", + ErrNoBinlog: "It is not allowed to shut off binlog on this command", + ErrConsecutiveReorgPartitions: "When reorganizing a set of partitions they must be in consecutive order", + ErrReorgOutsideRange: "Reorganize of range partitions cannot change total ranges except for last partition where it can extend the range", + ErrPartitionFunctionFailure: "Partition function not supported in this version for this handler", + ErrPartState: "Partition state cannot be defined from CREATE/ALTER TABLE", + ErrLimitedPartRange: "The %-.64s handler only supports 32 bit integers in VALUES", + ErrPluginIsNotLoaded: "Plugin '%-.192s' is not loaded", + ErrWrongValue: "Incorrect %-.32s value: '%-.128s'", + ErrNoPartitionForGivenValue: "Table has no partition for value %-.64s", + ErrFilegroupOptionOnlyOnce: "It is not allowed to specify %s more than once", + ErrCreateFilegroupFailed: "Failed to create %s", + ErrDropFilegroupFailed: "Failed to drop %s", + ErrTablespaceAutoExtend: "The handler doesn't support autoextend of tablespaces", + ErrWrongSizeNumber: "A size parameter was incorrectly specified, either number or on the form 10M", + ErrSizeOverflow: "The size number was correct but we don't allow the digit part to be more than 2 billion", + ErrAlterFilegroupFailed: "Failed to alter: %s", + ErrBinlogRowLoggingFailed: "Writing one row to the row-based binary log failed", + ErrBinlogRowWrongTableDef: "Table definition on master and slave does not match: %s", + ErrBinlogRowRbrToSbr: "Slave running with --log-slave-updates must use row-based binary logging to be able to replicate row-based binary log events", + ErrEventAlreadyExists: "Event '%-.192s' already exists", + ErrEventStoreFailed: "Failed to store event %s. Error code %d from storage engine.", + ErrEventDoesNotExist: "Unknown event '%-.192s'", + ErrEventCantAlter: "Failed to alter event '%-.192s'", + ErrEventDropFailed: "Failed to drop %s", + ErrEventIntervalNotPositiveOrTooBig: "INTERVAL is either not positive or too big", + ErrEventEndsBeforeStarts: "ENDS is either invalid or before STARTS", + ErrEventExecTimeInThePast: "Event execution time is in the past. Event has been disabled", + ErrEventOpenTableFailed: "Failed to open mysql.event", + ErrEventNeitherMExprNorMAt: "No datetime expression provided", + ErrObsoleteColCountDoesntMatchCorrupted: "Column count of mysql.%s is wrong. Expected %d, found %d. The table is probably corrupted", + ErrObsoleteCannotLoadFromTable: "Cannot load from mysql.%s. The table is probably corrupted", + ErrEventCannotDelete: "Failed to delete the event from mysql.event", + ErrEventCompile: "Error during compilation of event's body", + ErrEventSameName: "Same old and new event name", + ErrEventDataTooLong: "Data for column '%s' too long", + ErrDropIndexFk: "Cannot drop index '%-.192s': needed in a foreign key constraint", + ErrWarnDeprecatedSyntaxWithVer: "The syntax '%s' is deprecated and will be removed in MySQL %s. Please use %s instead", + ErrCantWriteLockLogTable: "You can't write-lock a log table. Only read access is possible", + ErrCantLockLogTable: "You can't use locks with log tables.", + ErrForeignDuplicateKeyOldUnused: "Upholding foreign key constraints for table '%.192s', entry '%-.192s', key %d would lead to a duplicate entry", + ErrColCountDoesntMatchPleaseUpdate: "Column count of mysql.%s is wrong. Expected %d, found %d. Created with MySQL %d, now running %d. Please use mysqlUpgrade to fix this error.", + ErrTempTablePreventsSwitchOutOfRbr: "Cannot switch out of the row-based binary log format when the session has open temporary tables", + ErrStoredFunctionPreventsSwitchBinlogFormat: "Cannot change the binary logging format inside a stored function or trigger", + ErrNdbCantSwitchBinlogFormat: "The NDB cluster engine does not support changing the binlog format on the fly yet", + ErrPartitionNoTemporary: "Cannot create temporary table with partitions", + ErrPartitionConstDomain: "Partition constant is out of partition function domain", + ErrPartitionFunctionIsNotAllowed: "This partition function is not allowed", + ErrDdlLog: "Error in DDL log", + ErrNullInValuesLessThan: "Not allowed to use NULL value in VALUES LESS THAN", + ErrWrongPartitionName: "Incorrect partition name", + ErrCantChangeTxCharacteristics: "Transaction characteristics can't be changed while a transaction is in progress", + ErrDupEntryAutoincrementCase: "ALTER TABLE causes autoIncrement resequencing, resulting in duplicate entry '%-.192s' for key '%-.192s'", + ErrEventModifyQueue: "Internal scheduler error %d", + ErrEventSetVar: "Error during starting/stopping of the scheduler. Error code %u", + ErrPartitionMerge: "Engine cannot be used in partitioned tables", + ErrCantActivateLog: "Cannot activate '%-.64s' log", + ErrRbrNotAvailable: "The server was not built with row-based replication", + ErrBase64Decode: "Decoding of base64 string failed", + ErrEventRecursionForbidden: "Recursion of EVENT DDL statements is forbidden when body is present", + ErrEventsDb: "Cannot proceed because system tables used by Event Scheduler were found damaged at server start", + ErrOnlyIntegersAllowed: "Only integers allowed as number here", + ErrUnsuportedLogEngine: "This storage engine cannot be used for log tables\"", + ErrBadLogStatement: "You cannot '%s' a log table if logging is enabled", + ErrCantRenameLogTable: "Cannot rename '%s'. When logging enabled, rename to/from log table must rename two tables: the log table to an archive table and another table back to '%s'", + ErrWrongParamcountToNativeFct: "Incorrect parameter count in the call to native function '%-.192s'", + ErrWrongParametersToNativeFct: "Incorrect parameters in the call to native function '%-.192s'", + ErrWrongParametersToStoredFct: "Incorrect parameters in the call to stored function '%-.192s'", + ErrNativeFctNameCollision: "This function '%-.192s' has the same name as a native function", + ErrDupEntryWithKeyName: "Duplicate entry '%-.64s' for key '%-.192s'", + ErrBinlogPurgeEmfile: "Too many files opened, please execute the command again", + ErrEventCannotCreateInThePast: "Event execution time is in the past and ON COMPLETION NOT PRESERVE is set. The event was dropped immediately after creation.", + ErrEventCannotAlterInThePast: "Event execution time is in the past and ON COMPLETION NOT PRESERVE is set. The event was not changed. Specify a time in the future.", + ErrSlaveIncident: "The incident %s occured on the master. Message: %-.64s", + ErrNoPartitionForGivenValueSilent: "Table has no partition for some existing values", + ErrBinlogUnsafeStatement: "Unsafe statement written to the binary log using statement format since BINLOGFORMAT = STATEMENT. %s", + ErrSlaveFatal: "Fatal : %s", + ErrSlaveRelayLogReadFailure: "Relay log read failure: %s", + ErrSlaveRelayLogWriteFailure: "Relay log write failure: %s", + ErrSlaveCreateEventFailure: "Failed to create %s", + ErrSlaveMasterComFailure: "Master command %s failed: %s", + ErrBinlogLoggingImpossible: "Binary logging not possible. Message: %s", + ErrViewNoCreationCtx: "View `%-.64s`.`%-.64s` has no creation context", + ErrViewInvalidCreationCtx: "Creation context of view `%-.64s`.`%-.64s' is invalid", + ErrSrInvalidCreationCtx: "Creation context of stored routine `%-.64s`.`%-.64s` is invalid", + ErrTrgCorruptedFile: "Corrupted TRG file for table `%-.64s`.`%-.64s`", + ErrTrgNoCreationCtx: "Triggers for table `%-.64s`.`%-.64s` have no creation context", + ErrTrgInvalidCreationCtx: "Trigger creation context of table `%-.64s`.`%-.64s` is invalid", + ErrEventInvalidCreationCtx: "Creation context of event `%-.64s`.`%-.64s` is invalid", + ErrTrgCantOpenTable: "Cannot open table for trigger `%-.64s`.`%-.64s`", + ErrCantCreateSroutine: "Cannot create stored routine `%-.64s`. Check warnings", + ErrNeverUsed: "Ambiguous slave modes combination. %s", + ErrNoFormatDescriptionEventBeforeBinlogStatement: "The BINLOG statement of type `%s` was not preceded by a format description BINLOG statement.", + ErrSlaveCorruptEvent: "Corrupted replication event was detected", + ErrLoadDataInvalidColumn: "Invalid column reference (%-.64s) in LOAD DATA", + ErrLogPurgeNoFile: "Being purged log %s was not found", + ErrXaRbtimeout: "XARBTIMEOUT: Transaction branch was rolled back: took too long", + ErrXaRbdeadlock: "XARBDEADLOCK: Transaction branch was rolled back: deadlock was detected", + ErrNeedReprepare: "Prepared statement needs to be re-prepared", + ErrDelayedNotSupported: "DELAYED option not supported for table '%-.192s'", + WarnNoMasterInfo: "The master info structure does not exist", + WarnOptionIgnored: "<%-.64s> option ignored", + WarnPluginDeleteBuiltin: "Built-in plugins cannot be deleted", + WarnPluginBusy: "Plugin is busy and will be uninstalled on shutdown", + ErrVariableIsReadonly: "%s variable '%s' is read-only. Use SET %s to assign the value", + ErrWarnEngineTransactionRollback: "Storage engine %s does not support rollback for this statement. Transaction rolled back and must be restarted", + ErrSlaveHeartbeatFailure: "Unexpected master's heartbeat data: %s", + ErrSlaveHeartbeatValueOutOfRange: "The requested value for the heartbeat period is either negative or exceeds the maximum allowed (%s seconds).", + ErrNdbReplicationSchema: "Bad schema for mysql.ndbReplication table. Message: %-.64s", + ErrConflictFnParse: "Error in parsing conflict function. Message: %-.64s", + ErrExceptionsWrite: "Write to exceptions table failed. Message: %-.128s\"", + ErrTooLongTableComment: "Comment for table '%-.64s' is too long (max = %lu)", + ErrTooLongFieldComment: "Comment for field '%-.64s' is too long (max = %lu)", + ErrFuncInexistentNameCollision: "FUNCTION %s does not exist. Check the 'Function Name Parsing and Resolution' section in the Reference Manual", + ErrDatabaseName: "Database", + ErrTableName: "Table", + ErrPartitionName: "Partition", + ErrSubpartitionName: "Subpartition", + ErrTemporaryName: "Temporary", + ErrRenamedName: "Renamed", + ErrTooManyConcurrentTrxs: "Too many active concurrent transactions", + WarnNonASCIISeparatorNotImplemented: "Non-ASCII separator arguments are not fully supported", + ErrDebugSyncTimeout: "debug sync point wait timed out", + ErrDebugSyncHitLimit: "debug sync point hit limit reached", + ErrDupSignalSet: "Duplicate condition information item '%s'", + ErrSignalWarn: "Unhandled user-defined warning condition", + ErrSignalNotFound: "Unhandled user-defined not found condition", + ErrSignalException: "Unhandled user-defined exception condition", + ErrResignalWithoutActiveHandler: "RESIGNAL when handler not active", + ErrSignalBadConditionType: "SIGNAL/RESIGNAL can only use a CONDITION defined with SQLSTATE", + WarnCondItemTruncated: "Data truncated for condition item '%s'", + ErrCondItemTooLong: "Data too long for condition item '%s'", + ErrUnknownLocale: "Unknown locale: '%-.64s'", + ErrSlaveIgnoreServerIds: "The requested server id %d clashes with the slave startup option --replicate-same-server-id", + ErrQueryCacheDisabled: "Query cache is disabled; restart the server with queryCacheType=1 to enable it", + ErrSameNamePartitionField: "Duplicate partition field name '%-.192s'", + ErrPartitionColumnList: "Inconsistency in usage of column lists for partitioning", + ErrWrongTypeColumnValue: "Partition column values of incorrect type", + ErrTooManyPartitionFuncFields: "Too many fields in '%-.192s'", + ErrMaxvalueInValuesIn: "Cannot use MAXVALUE as value in VALUES IN", + ErrTooManyValues: "Cannot have more than one value for this type of %-.64s partitioning", + ErrRowSinglePartitionField: "Row expressions in VALUES IN only allowed for multi-field column partitioning", + ErrFieldTypeNotAllowedAsPartitionField: "Field '%-.192s' is of a not allowed type for this type of partitioning", + ErrPartitionFieldsTooLong: "The total length of the partitioning fields is too large", + ErrBinlogRowEngineAndStmtEngine: "Cannot execute statement: impossible to write to binary log since both row-incapable engines and statement-incapable engines are involved.", + ErrBinlogRowModeAndStmtEngine: "Cannot execute statement: impossible to write to binary log since BINLOGFORMAT = ROW and at least one table uses a storage engine limited to statement-based logging.", + ErrBinlogUnsafeAndStmtEngine: "Cannot execute statement: impossible to write to binary log since statement is unsafe, storage engine is limited to statement-based logging, and BINLOGFORMAT = MIXED. %s", + ErrBinlogRowInjectionAndStmtEngine: "Cannot execute statement: impossible to write to binary log since statement is in row format and at least one table uses a storage engine limited to statement-based logging.", + ErrBinlogStmtModeAndRowEngine: "Cannot execute statement: impossible to write to binary log since BINLOGFORMAT = STATEMENT and at least one table uses a storage engine limited to row-based logging.%s", + ErrBinlogRowInjectionAndStmtMode: "Cannot execute statement: impossible to write to binary log since statement is in row format and BINLOGFORMAT = STATEMENT.", + ErrBinlogMultipleEnginesAndSelfLoggingEngine: "Cannot execute statement: impossible to write to binary log since more than one engine is involved and at least one engine is self-logging.", + ErrBinlogUnsafeLimit: "The statement is unsafe because it uses a LIMIT clause. This is unsafe because the set of rows included cannot be predicted.", + ErrBinlogUnsafeInsertDelayed: "The statement is unsafe because it uses INSERT DELAYED. This is unsafe because the times when rows are inserted cannot be predicted.", + ErrBinlogUnsafeSystemTable: "The statement is unsafe because it uses the general log, slow query log, or performanceSchema table(s). This is unsafe because system tables may differ on slaves.", + ErrBinlogUnsafeAutoincColumns: "Statement is unsafe because it invokes a trigger or a stored function that inserts into an AUTOINCREMENT column. Inserted values cannot be logged correctly.", + ErrBinlogUnsafeUdf: "Statement is unsafe because it uses a UDF which may not return the same value on the slave.", + ErrBinlogUnsafeSystemVariable: "Statement is unsafe because it uses a system variable that may have a different value on the slave.", + ErrBinlogUnsafeSystemFunction: "Statement is unsafe because it uses a system function that may return a different value on the slave.", + ErrBinlogUnsafeNontransAfterTrans: "Statement is unsafe because it accesses a non-transactional table after accessing a transactional table within the same transaction.", + ErrMessageAndStatement: "%s Statement: %s", + ErrSlaveConversionFailed: "Column %d of table '%-.192s.%-.192s' cannot be converted from type '%-.32s' to type '%-.32s'", + ErrSlaveCantCreateConversion: "Can't create conversion table for table '%-.192s.%-.192s'", + ErrInsideTransactionPreventsSwitchBinlogFormat: "Cannot modify @@session.binlogFormat inside a transaction", + ErrPathLength: "The path specified for %.64s is too long.", + ErrWarnDeprecatedSyntaxNoReplacement: "'%s' is deprecated and will be removed in a future release.", + ErrWrongNativeTableStructure: "Native table '%-.64s'.'%-.64s' has the wrong structure", + ErrWrongPerfschemaUsage: "Invalid performanceSchema usage.", + ErrWarnISSkippedTable: "Table '%s'.'%s' was skipped since its definition is being modified by concurrent DDL statement", + ErrInsideTransactionPreventsSwitchBinlogDirect: "Cannot modify @@session.binlogDirectNonTransactionalUpdates inside a transaction", + ErrStoredFunctionPreventsSwitchBinlogDirect: "Cannot change the binlog direct flag inside a stored function or trigger", + ErrSpatialMustHaveGeomCol: "A SPATIAL index may only contain a geometrical type column", + ErrTooLongIndexComment: "Comment for index '%-.64s' is too long (max = %lu)", + ErrLockAborted: "Wait on a lock was aborted due to a pending exclusive lock", + ErrDataOutOfRange: "%s value is out of range in '%s'", + ErrWrongSpvarTypeInLimit: "A variable of a non-integer based type in LIMIT clause", + ErrBinlogUnsafeMultipleEnginesAndSelfLoggingEngine: "Mixing self-logging and non-self-logging engines in a statement is unsafe.", + ErrBinlogUnsafeMixedStatement: "Statement accesses nontransactional table as well as transactional or temporary table, and writes to any of them.", + ErrInsideTransactionPreventsSwitchSQLLogBin: "Cannot modify @@session.sqlLogBin inside a transaction", + ErrStoredFunctionPreventsSwitchSQLLogBin: "Cannot change the sqlLogBin inside a stored function or trigger", + ErrFailedReadFromParFile: "Failed to read from the .par file", + ErrValuesIsNotIntType: "VALUES value for partition '%-.64s' must have type INT", + ErrAccessDeniedNoPassword: "Access denied for user '%-.48s'@'%-.64s'", + ErrSetPasswordAuthPlugin: "SET PASSWORD has no significance for users authenticating via plugins", + ErrGrantPluginUserExists: "GRANT with IDENTIFIED WITH is illegal because the user %-.*s already exists", + ErrTruncateIllegalFk: "Cannot truncate a table referenced in a foreign key constraint (%.192s)", + ErrPluginIsPermanent: "Plugin '%s' is forcePlusPermanent and can not be unloaded", + ErrSlaveHeartbeatValueOutOfRangeMin: "The requested value for the heartbeat period is less than 1 millisecond. The value is reset to 0, meaning that heartbeating will effectively be disabled.", + ErrSlaveHeartbeatValueOutOfRangeMax: "The requested value for the heartbeat period exceeds the value of `slaveNetTimeout' seconds. A sensible value for the period should be less than the timeout.", + ErrStmtCacheFull: "Multi-row statements required more than 'maxBinlogStmtCacheSize' bytes of storage; increase this mysqld variable and try again", + ErrMultiUpdateKeyConflict: "Primary key/partition key update is not allowed since the table is updated both as '%-.192s' and '%-.192s'.", + ErrTableNeedsRebuild: "Table rebuild required. Please do \"ALTER TABLE `%-.32s` FORCE\" or dump/reload to fix it!", + WarnOptionBelowLimit: "The value of '%s' should be no less than the value of '%s'", + ErrIndexColumnTooLong: "Index column size too large. The maximum column size is %lu bytes.", + ErrErrorInTriggerBody: "Trigger '%-.64s' has an error in its body: '%-.256s'", + ErrErrorInUnknownTriggerBody: "Unknown trigger has an error in its body: '%-.256s'", + ErrIndexCorrupt: "Index %s is corrupted", + ErrUndoRecordTooBig: "Undo log record is too big.", + ErrBinlogUnsafeInsertIgnoreSelect: "INSERT IGNORE... SELECT is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are ignored. This order cannot be predicted and may differ on master and the slave.", + ErrBinlogUnsafeInsertSelectUpdate: "INSERT... SELECT... ON DUPLICATE KEY UPDATE is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are updated. This order cannot be predicted and may differ on master and the slave.", + ErrBinlogUnsafeReplaceSelect: "REPLACE... SELECT is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are replaced. This order cannot be predicted and may differ on master and the slave.", + ErrBinlogUnsafeCreateIgnoreSelect: "CREATE... IGNORE SELECT is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are ignored. This order cannot be predicted and may differ on master and the slave.", + ErrBinlogUnsafeCreateReplaceSelect: "CREATE... REPLACE SELECT is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are replaced. This order cannot be predicted and may differ on master and the slave.", + ErrBinlogUnsafeUpdateIgnore: "UPDATE IGNORE is unsafe because the order in which rows are updated determines which (if any) rows are ignored. This order cannot be predicted and may differ on master and the slave.", + ErrPluginNoUninstall: "Plugin '%s' is marked as not dynamically uninstallable. You have to stop the server to uninstall it.", + ErrPluginNoInstall: "Plugin '%s' is marked as not dynamically installable. You have to stop the server to install it.", + ErrBinlogUnsafeWriteAutoincSelect: "Statements writing to a table with an auto-increment column after selecting from another table are unsafe because the order in which rows are retrieved determines what (if any) rows will be written. This order cannot be predicted and may differ on master and the slave.", + ErrBinlogUnsafeCreateSelectAutoinc: "CREATE TABLE... SELECT... on a table with an auto-increment column is unsafe because the order in which rows are retrieved by the SELECT determines which (if any) rows are inserted. This order cannot be predicted and may differ on master and the slave.", + ErrBinlogUnsafeInsertTwoKeys: "INSERT... ON DUPLICATE KEY UPDATE on a table with more than one UNIQUE KEY is unsafe", + ErrTableInFkCheck: "Table is being used in foreign key check.", + ErrUnsupportedEngine: "Storage engine '%s' does not support system tables. [%s.%s]", + ErrBinlogUnsafeAutoincNotFirst: "INSERT into autoincrement field which is not the first part in the composed primary key is unsafe.", + ErrCannotLoadFromTableV2: "Cannot load from %s.%s. The table is probably corrupted", + ErrMasterDelayValueOutOfRange: "The requested value %u for the master delay exceeds the maximum %u", + ErrOnlyFdAndRbrEventsAllowedInBinlogStatement: "Only FormatDescriptionLogEvent and row events are allowed in BINLOG statements (but %s was provided)", + ErrPartitionExchangeDifferentOption: "Non matching attribute '%-.64s' between partition and table", + ErrPartitionExchangePartTable: "Table to exchange with partition is partitioned: '%-.64s'", + ErrPartitionExchangeTempTable: "Table to exchange with partition is temporary: '%-.64s'", + ErrPartitionInsteadOfSubpartition: "Subpartitioned table, use subpartition instead of partition", + ErrUnknownPartition: "Unknown partition '%-.64s' in table '%-.64s'", + ErrTablesDifferentMetadata: "Tables have different definitions", + ErrRowDoesNotMatchPartition: "Found a row that does not match the partition", + ErrBinlogCacheSizeGreaterThanMax: "Option binlogCacheSize (%lu) is greater than maxBinlogCacheSize (%lu); setting binlogCacheSize equal to maxBinlogCacheSize.", + ErrWarnIndexNotApplicable: "Cannot use %-.64s access on index '%-.64s' due to type or collation conversion on field '%-.64s'", + ErrPartitionExchangeForeignKey: "Table to exchange with partition has foreign key references: '%-.64s'", + ErrNoSuchKeyValue: "Key value '%-.192s' was not found in table '%-.192s.%-.192s'", + ErrRplInfoDataTooLong: "Data for column '%s' too long", + ErrNetworkReadEventChecksumFailure: "Replication event checksum verification failed while reading from network.", + ErrBinlogReadEventChecksumFailure: "Replication event checksum verification failed while reading from a log file.", + ErrBinlogStmtCacheSizeGreaterThanMax: "Option binlogStmtCacheSize (%lu) is greater than maxBinlogStmtCacheSize (%lu); setting binlogStmtCacheSize equal to maxBinlogStmtCacheSize.", + ErrCantUpdateTableInCreateTableSelect: "Can't update table '%-.192s' while '%-.192s' is being created.", + ErrPartitionClauseOnNonpartitioned: "PARTITION () clause on non partitioned table", + ErrRowDoesNotMatchGivenPartitionSet: "Found a row not matching the given partition set", + ErrNoSuchPartitionunused: "partition '%-.64s' doesn't exist", + ErrChangeRplInfoRepositoryFailure: "Failure while changing the type of replication repository: %s.", + ErrWarningNotCompleteRollbackWithCreatedTempTable: "The creation of some temporary tables could not be rolled back.", + ErrWarningNotCompleteRollbackWithDroppedTempTable: "Some temporary tables were dropped, but these operations could not be rolled back.", + ErrMtsFeatureIsNotSupported: "%s is not supported in multi-threaded slave mode. %s", + ErrMtsUpdatedDbsGreaterMax: "The number of modified databases exceeds the maximum %d; the database names will not be included in the replication event metadata.", + ErrMtsCantParallel: "Cannot execute the current event group in the parallel mode. Encountered event %s, relay-log name %s, position %s which prevents execution of this event group in parallel mode. Reason: %s.", + ErrMtsInconsistentData: "%s", + ErrFulltextNotSupportedWithPartitioning: "FULLTEXT index is not supported for partitioned tables.", + ErrDaInvalidConditionNumber: "Invalid condition number", + ErrInsecurePlainText: "Sending passwords in plain text without SSL/TLS is extremely insecure.", + ErrInsecureChangeMaster: "Storing MySQL user name or password information in the master.info repository is not secure and is therefore not recommended. Please see the MySQL Manual for more about this issue and possible alternatives.", + ErrForeignDuplicateKeyWithChildInfo: "Foreign key constraint for table '%.192s', record '%-.192s' would lead to a duplicate entry in table '%.192s', key '%.192s'", + ErrForeignDuplicateKeyWithoutChildInfo: "Foreign key constraint for table '%.192s', record '%-.192s' would lead to a duplicate entry in a child table", + ErrSQLthreadWithSecureSlave: "Setting authentication options is not possible when only the Slave SQL Thread is being started.", + ErrTableHasNoFt: "The table does not have FULLTEXT index to support this query", + ErrVariableNotSettableInSfOrTrigger: "The system variable %.200s cannot be set in stored functions or triggers.", + ErrVariableNotSettableInTransaction: "The system variable %.200s cannot be set when there is an ongoing transaction.", + ErrGtidNextIsNotInGtidNextList: "The system variable @@SESSION.GTIDNEXT has the value %.200s, which is not listed in @@SESSION.GTIDNEXTLIST.", + ErrCantChangeGtidNextInTransactionWhenGtidNextListIsNull: "When @@SESSION.GTIDNEXTLIST == NULL, the system variable @@SESSION.GTIDNEXT cannot change inside a transaction.", + ErrSetStatementCannotInvokeFunction: "The statement 'SET %.200s' cannot invoke a stored function.", + ErrGtidNextCantBeAutomaticIfGtidNextListIsNonNull: "The system variable @@SESSION.GTIDNEXT cannot be 'AUTOMATIC' when @@SESSION.GTIDNEXTLIST is non-NULL.", + ErrSkippingLoggedTransaction: "Skipping transaction %.200s because it has already been executed and logged.", + ErrMalformedGtidSetSpecification: "Malformed GTID set specification '%.200s'.", + ErrMalformedGtidSetEncoding: "Malformed GTID set encoding.", + ErrMalformedGtidSpecification: "Malformed GTID specification '%.200s'.", + ErrGnoExhausted: "Impossible to generate Global Transaction Identifier: the integer component reached the maximal value. Restart the server with a new serverUuid.", + ErrBadSlaveAutoPosition: "Parameters MASTERLOGFILE, MASTERLOGPOS, RELAYLOGFILE and RELAYLOGPOS cannot be set when MASTERAUTOPOSITION is active.", + ErrAutoPositionRequiresGtidModeOn: "CHANGE MASTER TO MASTERAUTOPOSITION = 1 can only be executed when @@GLOBAL.GTIDMODE = ON.", + ErrCantDoImplicitCommitInTrxWhenGtidNextIsSet: "Cannot execute statements with implicit commit inside a transaction when @@SESSION.GTIDNEXT != AUTOMATIC or @@SESSION.GTIDNEXTLIST != NULL.", + ErrGtidMode2Or3RequiresEnforceGtidConsistencyOn: "@@GLOBAL.GTIDMODE = ON or UPGRADESTEP2 requires @@GLOBAL.ENFORCEGTIDCONSISTENCY = 1.", + ErrGtidModeRequiresBinlog: "@@GLOBAL.GTIDMODE = ON or UPGRADESTEP1 or UPGRADESTEP2 requires --log-bin and --log-slave-updates.", + ErrCantSetGtidNextToGtidWhenGtidModeIsOff: "@@SESSION.GTIDNEXT cannot be set to UUID:NUMBER when @@GLOBAL.GTIDMODE = OFF.", + ErrCantSetGtidNextToAnonymousWhenGtidModeIsOn: "@@SESSION.GTIDNEXT cannot be set to ANONYMOUS when @@GLOBAL.GTIDMODE = ON.", + ErrCantSetGtidNextListToNonNullWhenGtidModeIsOff: "@@SESSION.GTIDNEXTLIST cannot be set to a non-NULL value when @@GLOBAL.GTIDMODE = OFF.", + ErrFoundGtidEventWhenGtidModeIsOff: "Found a GtidLogEvent or PreviousGtidsLogEvent when @@GLOBAL.GTIDMODE = OFF.", + ErrGtidUnsafeNonTransactionalTable: "When @@GLOBAL.ENFORCEGTIDCONSISTENCY = 1, updates to non-transactional tables can only be done in either autocommitted statements or single-statement transactions, and never in the same statement as updates to transactional tables.", + ErrGtidUnsafeCreateSelect: "CREATE TABLE ... SELECT is forbidden when @@GLOBAL.ENFORCEGTIDCONSISTENCY = 1.", + ErrGtidUnsafeCreateDropTemporaryTableInTransaction: "When @@GLOBAL.ENFORCEGTIDCONSISTENCY = 1, the statements CREATE TEMPORARY TABLE and DROP TEMPORARY TABLE can be executed in a non-transactional context only, and require that AUTOCOMMIT = 1.", + ErrGtidModeCanOnlyChangeOneStepAtATime: "The value of @@GLOBAL.GTIDMODE can only change one step at a time: OFF <-> UPGRADESTEP1 <-> UPGRADESTEP2 <-> ON. Also note that this value must be stepped up or down simultaneously on all servers; see the Manual for instructions.", + ErrMasterHasPurgedRequiredGtids: "The slave is connecting using CHANGE MASTER TO MASTERAUTOPOSITION = 1, but the master has purged binary logs containing GTIDs that the slave requires.", + ErrCantSetGtidNextWhenOwningGtid: "@@SESSION.GTIDNEXT cannot be changed by a client that owns a GTID. The client owns %s. Ownership is released on COMMIT or ROLLBACK.", + ErrUnknownExplainFormat: "Unknown EXPLAIN format name: '%s'", + ErrCantExecuteInReadOnlyTransaction: "Cannot execute statement in a READ ONLY transaction.", + ErrTooLongTablePartitionComment: "Comment for table partition '%-.64s' is too long (max = %lu)", + ErrSlaveConfiguration: "Slave is not configured or failed to initialize properly. You must at least set --server-id to enable either a master or a slave. Additional error messages can be found in the MySQL error log.", + ErrInnodbFtLimit: "InnoDB presently supports one FULLTEXT index creation at a time", + ErrInnodbNoFtTempTable: "Cannot create FULLTEXT index on temporary InnoDB table", + ErrInnodbFtWrongDocidColumn: "Column '%-.192s' is of wrong type for an InnoDB FULLTEXT index", + ErrInnodbFtWrongDocidIndex: "Index '%-.192s' is of wrong type for an InnoDB FULLTEXT index", + ErrInnodbOnlineLogTooBig: "Creating index '%-.192s' required more than 'innodbOnlineAlterLogMaxSize' bytes of modification log. Please try again.", + ErrUnknownAlterAlgorithm: "Unknown ALGORITHM '%s'", + ErrUnknownAlterLock: "Unknown LOCK type '%s'", + ErrMtsChangeMasterCantRunWithGaps: "CHANGE MASTER cannot be executed when the slave was stopped with an error or killed in MTS mode. Consider using RESET SLAVE or START SLAVE UNTIL.", + ErrMtsRecoveryFailure: "Cannot recover after SLAVE errored out in parallel execution mode. Additional error messages can be found in the MySQL error log.", + ErrMtsResetWorkers: "Cannot clean up worker info tables. Additional error messages can be found in the MySQL error log.", + ErrColCountDoesntMatchCorruptedV2: "Column count of %s.%s is wrong. Expected %d, found %d. The table is probably corrupted", + ErrSlaveSilentRetryTransaction: "Slave must silently retry current transaction", + ErrDiscardFkChecksRunning: "There is a foreign key check running on table '%-.192s'. Cannot discard the table.", + ErrTableSchemaMismatch: "Schema mismatch (%s)", + ErrTableInSystemTablespace: "Table '%-.192s' in system tablespace", + ErrIoRead: "IO Read : (%lu, %s) %s", + ErrIoWrite: "IO Write : (%lu, %s) %s", + ErrTablespaceMissing: "Tablespace is missing for table '%-.192s'", + ErrTablespaceExists: "Tablespace for table '%-.192s' exists. Please DISCARD the tablespace before IMPORT.", + ErrTablespaceDiscarded: "Tablespace has been discarded for table '%-.192s'", + ErrInternal: "Internal : %s", + ErrInnodbImport: "ALTER TABLE '%-.192s' IMPORT TABLESPACE failed with error %lu : '%s'", + ErrInnodbIndexCorrupt: "Index corrupt: %s", + ErrInvalidYearColumnLength: "YEAR(%lu) column type is deprecated. Creating YEAR(4) column instead.", + ErrNotValidPassword: "Your password does not satisfy the current policy requirements", + ErrMustChangePassword: "You must SET PASSWORD before executing this statement", + ErrFkNoIndexChild: "Failed to add the foreign key constaint. Missing index for constraint '%s' in the foreign table '%s'", + ErrFkNoIndexParent: "Failed to add the foreign key constaint. Missing index for constraint '%s' in the referenced table '%s'", + ErrFkFailAddSystem: "Failed to add the foreign key constraint '%s' to system tables", + ErrFkCannotOpenParent: "Failed to open the referenced table '%s'", + ErrFkIncorrectOption: "Failed to add the foreign key constraint on table '%s'. Incorrect options in FOREIGN KEY constraint '%s'", + ErrFkDupName: "Duplicate foreign key constraint name '%s'", + ErrPasswordFormat: "The password hash doesn't have the expected format. Check if the correct password algorithm is being used with the PASSWORD() function.", + ErrFkColumnCannotDrop: "Cannot drop column '%-.192s': needed in a foreign key constraint '%-.192s'", + ErrFkColumnCannotDropChild: "Cannot drop column '%-.192s': needed in a foreign key constraint '%-.192s' of table '%-.192s'", + ErrFkColumnNotNull: "Column '%-.192s' cannot be NOT NULL: needed in a foreign key constraint '%-.192s' SET NULL", + ErrDupIndex: "Duplicate index '%-.64s' defined on the table '%-.64s.%-.64s'. This is deprecated and will be disallowed in a future release.", + ErrFkColumnCannotChange: "Cannot change column '%-.192s': used in a foreign key constraint '%-.192s'", + ErrFkColumnCannotChangeChild: "Cannot change column '%-.192s': used in a foreign key constraint '%-.192s' of table '%-.192s'", + ErrFkCannotDeleteParent: "Cannot delete rows from table which is parent in a foreign key constraint '%-.192s' of table '%-.192s'", + ErrMalformedPacket: "Malformed communication packet.", + ErrReadOnlyMode: "Running in read-only mode", + ErrGtidNextTypeUndefinedGroup: "When @@SESSION.GTIDNEXT is set to a GTID, you must explicitly set it again after a COMMIT or ROLLBACK. If you see this error message in the slave SQL thread, it means that a table in the current transaction is transactional on the master and non-transactional on the slave. In a client connection, it means that you executed SET @@SESSION.GTIDNEXT before a transaction and forgot to set @@SESSION.GTIDNEXT to a different identifier or to 'AUTOMATIC' after COMMIT or ROLLBACK. Current @@SESSION.GTIDNEXT is '%s'.", + ErrVariableNotSettableInSp: "The system variable %.200s cannot be set in stored procedures.", + ErrCantSetGtidPurgedWhenGtidModeIsOff: "@@GLOBAL.GTIDPURGED can only be set when @@GLOBAL.GTIDMODE = ON.", + ErrCantSetGtidPurgedWhenGtidExecutedIsNotEmpty: "@@GLOBAL.GTIDPURGED can only be set when @@GLOBAL.GTIDEXECUTED is empty.", + ErrCantSetGtidPurgedWhenOwnedGtidsIsNotEmpty: "@@GLOBAL.GTIDPURGED can only be set when there are no ongoing transactions (not even in other clients).", + ErrGtidPurgedWasChanged: "@@GLOBAL.GTIDPURGED was changed from '%s' to '%s'.", + ErrGtidExecutedWasChanged: "@@GLOBAL.GTIDEXECUTED was changed from '%s' to '%s'.", + ErrBinlogStmtModeAndNoReplTables: "Cannot execute statement: impossible to write to binary log since BINLOGFORMAT = STATEMENT, and both replicated and non replicated tables are written to.", + ErrAlterOperationNotSupported: "%s is not supported for this operation. Try %s.", + ErrAlterOperationNotSupportedReason: "%s is not supported. Reason: %s. Try %s.", + ErrAlterOperationNotSupportedReasonCopy: "COPY algorithm requires a lock", + ErrAlterOperationNotSupportedReasonPartition: "Partition specific operations do not yet support LOCK/ALGORITHM", + ErrAlterOperationNotSupportedReasonFkRename: "Columns participating in a foreign key are renamed", + ErrAlterOperationNotSupportedReasonColumnType: "Cannot change column type INPLACE", + ErrAlterOperationNotSupportedReasonFkCheck: "Adding foreign keys needs foreignKeyChecks=OFF", + ErrAlterOperationNotSupportedReasonIgnore: "Creating unique indexes with IGNORE requires COPY algorithm to remove duplicate rows", + ErrAlterOperationNotSupportedReasonNopk: "Dropping a primary key is not allowed without also adding a new primary key", + ErrAlterOperationNotSupportedReasonAutoinc: "Adding an auto-increment column requires a lock", + ErrAlterOperationNotSupportedReasonHiddenFts: "Cannot replace hidden FTSDOCID with a user-visible one", + ErrAlterOperationNotSupportedReasonChangeFts: "Cannot drop or rename FTSDOCID", + ErrAlterOperationNotSupportedReasonFts: "Fulltext index creation requires a lock", + ErrSQLSlaveSkipCounterNotSettableInGtidMode: "sqlSlaveSkipCounter can not be set when the server is running with @@GLOBAL.GTIDMODE = ON. Instead, for each transaction that you want to skip, generate an empty transaction with the same GTID as the transaction", + ErrDupUnknownInIndex: "Duplicate entry for key '%-.192s'", + ErrIdentCausesTooLongPath: "Long database name and identifier for object resulted in path length exceeding %d characters. Path: '%s'.", + ErrAlterOperationNotSupportedReasonNotNull: "cannot silently convert NULL values, as required in this SQLMODE", + ErrMustChangePasswordLogin: "Your password has expired. To log in you must change it using a client that supports expired passwords.", + ErrRowInWrongPartition: "Found a row in wrong partition %s", } diff --git a/mysqldef/error.go b/mysqldef/error.go index d947327248..6b75670fc8 100644 --- a/mysqldef/error.go +++ b/mysqldef/error.go @@ -36,9 +36,8 @@ func (e *SQLError) Error() string { return fmt.Sprintf("ERROR %d (%s): %s", e.Code, e.State, e.Message) } -// NewDefaultError generates a SQL error, with an error code and -// extra arguments for a message format specifier. -func NewDefaultError(errCode uint16, args ...interface{}) *SQLError { +// NewErr generates a SQL error, with an error code and default format specifier defined in MySQLErrName. +func NewErr(errCode uint16, args ...interface{}) *SQLError { e := &SQLError{Code: errCode} if s, ok := MySQLState[errCode]; ok { @@ -56,8 +55,8 @@ func NewDefaultError(errCode uint16, args ...interface{}) *SQLError { return e } -// NewError creates a SQL error, with an error code and error details. -func NewError(errCode uint16, message string) *SQLError { +// NewErrf creates a SQL error, with an error code and a format specifier +func NewErrf(errCode uint16, format string, args ...interface{}) *SQLError { e := &SQLError{Code: errCode} if s, ok := MySQLState[errCode]; ok { @@ -66,7 +65,7 @@ func NewError(errCode uint16, message string) *SQLError { e.State = DefaultMySQLState } - e.Message = message + e.Message = fmt.Sprintf(format, args...) return e } diff --git a/mysqldef/error_test.go b/mysqldef/error_test.go index eb8a054a6c..10f9dd2d8e 100644 --- a/mysqldef/error_test.go +++ b/mysqldef/error_test.go @@ -23,15 +23,15 @@ type testSQLErrorSuite struct { } func (s *testSQLErrorSuite) TestSQLError(c *C) { - e := NewError(ErNoDbError, "no db error") + e := NewErrf(ErrNoDb, "no db error") c.Assert(len(e.Error()), Greater, 0) - e = NewError(0, "customized error") + e = NewErrf(0, "customized error") c.Assert(len(e.Error()), Greater, 0) - e = NewDefaultError(ErNoDbError) + e = NewErr(ErrNoDb) c.Assert(len(e.Error()), Greater, 0) - e = NewDefaultError(0, "customized error") + e = NewErr(0, "customized error") c.Assert(len(e.Error()), Greater, 0) } diff --git a/mysqldef/state.go b/mysqldef/state.go index 82c3694d23..9dfb4e9c14 100644 --- a/mysqldef/state.go +++ b/mysqldef/state.go @@ -21,229 +21,229 @@ const ( // MySQLState maps error code to MySQL SQLSTATE value. // The values are taken from ANSI SQL and ODBC and are more standardized. var MySQLState = map[uint16]string{ - ErDupKey: "23000", - ErOutofmemory: "HY001", - ErOutOfSortmemory: "HY001", - ErConCountError: "08004", - ErBadHostError: "08S01", - ErHandshakeError: "08S01", - ErDbaccessDeniedError: "42000", - ErAccessDeniedError: "28000", - ErNoDbError: "3D000", - ErUnknownComError: "08S01", - ErBadNullError: "23000", - ErBadDbError: "42000", - ErTableExistsError: "42S01", - ErBadTableError: "42S02", - ErNonUniqError: "23000", - ErServerShutdown: "08S01", - ErBadFieldError: "42S22", - ErWrongFieldWithGroup: "42000", - ErWrongSumSelect: "42000", - ErWrongGroupField: "42000", - ErWrongValueCount: "21S01", - ErTooLongIdent: "42000", - ErDupFieldname: "42S21", - ErDupKeyname: "42000", - ErDupEntry: "23000", - ErWrongFieldSpec: "42000", - ErParseError: "42000", - ErEmptyQuery: "42000", - ErNonuniqTable: "42000", - ErInvalidDefault: "42000", - ErMultiplePriKey: "42000", - ErTooManyKeys: "42000", - ErTooManyKeyParts: "42000", - ErTooLongKey: "42000", - ErKeyColumnDoesNotExits: "42000", - ErBlobUsedAsKey: "42000", - ErTooBigFieldlength: "42000", - ErWrongAutoKey: "42000", - ErForcingClose: "08S01", - ErIpsockError: "08S01", - ErNoSuchIndex: "42S12", - ErWrongFieldTerminators: "42000", - ErBlobsAndNoTerminated: "42000", - ErCantRemoveAllFields: "42000", - ErCantDropFieldOrKey: "42000", - ErBlobCantHaveDefault: "42000", - ErWrongDbName: "42000", - ErWrongTableName: "42000", - ErTooBigSelect: "42000", - ErUnknownProcedure: "42000", - ErWrongParamcountToProcedure: "42000", - ErUnknownTable: "42S02", - ErFieldSpecifiedTwice: "42000", - ErUnsupportedExtension: "42000", - ErTableMustHaveColumns: "42000", - ErUnknownCharacterSet: "42000", - ErTooBigRowsize: "42000", - ErWrongOuterJoin: "42000", - ErNullColumnInIndex: "42000", - ErPasswordAnonymousUser: "42000", - ErPasswordNotAllowed: "42000", - ErPasswordNoMatch: "42000", - ErWrongValueCountOnRow: "21S01", - ErInvalidUseOfNull: "22004", - ErRegexpError: "42000", - ErMixOfGroupFuncAndFields: "42000", - ErNonexistingGrant: "42000", - ErTableaccessDeniedError: "42000", - ErColumnaccessDeniedError: "42000", - ErIllegalGrantForTable: "42000", - ErGrantWrongHostOrUser: "42000", - ErNoSuchTable: "42S02", - ErNonexistingTableGrant: "42000", - ErNotAllowedCommand: "42000", - ErSyntaxError: "42000", - ErAbortingConnection: "08S01", - ErNetPacketTooLarge: "08S01", - ErNetReadErrorFromPipe: "08S01", - ErNetFcntlError: "08S01", - ErNetPacketsOutOfOrder: "08S01", - ErNetUncompressError: "08S01", - ErNetReadError: "08S01", - ErNetReadInterrupted: "08S01", - ErNetErrorOnWrite: "08S01", - ErNetWriteInterrupted: "08S01", - ErTooLongString: "42000", - ErTableCantHandleBlob: "42000", - ErTableCantHandleAutoIncrement: "42000", - ErWrongColumnName: "42000", - ErWrongKeyColumn: "42000", - ErDupUnique: "23000", - ErBlobKeyWithoutLength: "42000", - ErPrimaryCantHaveNull: "42000", - ErTooManyRows: "42000", - ErRequiresPrimaryKey: "42000", - ErKeyDoesNotExits: "42000", - ErCheckNoSuchTable: "42000", - ErCheckNotImplemented: "42000", - ErCantDoThisDuringAnTransaction: "25000", - ErNewAbortingConnection: "08S01", - ErMasterNetRead: "08S01", - ErMasterNetWrite: "08S01", - ErTooManyUserConnections: "42000", - ErReadOnlyTransaction: "25000", - ErNoPermissionToCreateUser: "42000", - ErLockDeadlock: "40001", - ErNoReferencedRow: "23000", - ErRowIsReferenced: "23000", - ErConnectToMaster: "08S01", - ErWrongNumberOfColumnsInSelect: "21000", - ErUserLimitReached: "42000", - ErSpecificAccessDeniedError: "42000", - ErNoDefault: "42000", - ErWrongValueForVar: "42000", - ErWrongTypeForVar: "42000", - ErCantUseOptionHere: "42000", - ErNotSupportedYet: "42000", - ErWrongFkDef: "42000", - ErOperandColumns: "21000", - ErSubqueryNo1Row: "21000", - ErIllegalReference: "42S22", - ErDerivedMustHaveAlias: "42000", - ErSelectReduced: "01000", - ErTablenameNotAllowedHere: "42000", - ErNotSupportedAuthMode: "08004", - ErSpatialCantHaveNull: "42000", - ErCollationCharsetMismatch: "42000", - ErWarnTooFewRecords: "01000", - ErWarnTooManyRecords: "01000", - ErWarnNullToNotnull: "22004", - ErWarnDataOutOfRange: "22003", - WarnDataTruncated: "01000", - ErWrongNameForIndex: "42000", - ErWrongNameForCatalog: "42000", - ErUnknownStorageEngine: "42000", - ErTruncatedWrongValue: "22007", - ErSpNoRecursiveCreate: "2F003", - ErSpAlreadyExists: "42000", - ErSpDoesNotExist: "42000", - ErSpLilabelMismatch: "42000", - ErSpLabelRedefine: "42000", - ErSpLabelMismatch: "42000", - ErSpUninitVar: "01000", - ErSpBadselect: "0A000", - ErSpBadreturn: "42000", - ErSpBadstatement: "0A000", - ErUpdateLogDeprecatedIgnored: "42000", - ErUpdateLogDeprecatedTranslated: "42000", - ErQueryInterrupted: "70100", - ErSpWrongNoOfArgs: "42000", - ErSpCondMismatch: "42000", - ErSpNoreturn: "42000", - ErSpNoreturnend: "2F005", - ErSpBadCursorQuery: "42000", - ErSpBadCursorSelect: "42000", - ErSpCursorMismatch: "42000", - ErSpCursorAlreadyOpen: "24000", - ErSpCursorNotOpen: "24000", - ErSpUndeclaredVar: "42000", - ErSpFetchNoData: "02000", - ErSpDupParam: "42000", - ErSpDupVar: "42000", - ErSpDupCond: "42000", - ErSpDupCurs: "42000", - ErSpSubselectNyi: "0A000", - ErStmtNotAllowedInSfOrTrg: "0A000", - ErSpVarcondAfterCurshndlr: "42000", - ErSpCursorAfterHandler: "42000", - ErSpCaseNotFound: "20000", - ErDivisionByZero: "22012", - ErIllegalValueForType: "22007", - ErProcaccessDeniedError: "42000", - ErXaerNota: "XAE04", - ErXaerInval: "XAE05", - ErXaerRmfail: "XAE07", - ErXaerOutside: "XAE09", - ErXaerRmerr: "XAE03", - ErXaRbrollback: "XA100", - ErNonexistingProcGrant: "42000", - ErDataTooLong: "22001", - ErSpBadSQLstate: "42000", - ErCantCreateUserWithGrant: "42000", - ErSpDupHandler: "42000", - ErSpNotVarArg: "42000", - ErSpNoRetset: "0A000", - ErCantCreateGeometryObject: "22003", - ErTooBigScale: "42000", - ErTooBigPrecision: "42000", - ErMBiggerThanD: "42000", - ErTooLongBody: "42000", - ErTooBigDisplaywidth: "42000", - ErXaerDupid: "XAE08", - ErDatetimeFunctionOverflow: "22008", - ErRowIsReferenced2: "23000", - ErNoReferencedRow2: "23000", - ErSpBadVarShadow: "42000", - ErSpWrongName: "42000", - ErSpNoAggregate: "42000", - ErMaxPreparedStmtCountReached: "42000", - ErNonGroupingFieldUsed: "42000", - ErForeignDuplicateKeyOldUnused: "23000", - ErCantChangeTxCharacteristics: "25001", - ErWrongParamcountToNativeFct: "42000", - ErWrongParametersToNativeFct: "42000", - ErWrongParametersToStoredFct: "42000", - ErDupEntryWithKeyName: "23000", - ErXaRbtimeout: "XA106", - ErXaRbdeadlock: "XA102", - ErFuncInexistentNameCollision: "42000", - ErDupSignalSet: "42000", - ErSignalWarn: "01000", - ErSignalNotFound: "02000", - ErSignalException: "HY000", - ErResignalWithoutActiveHandler: "0K000", - ErSpatialMustHaveGeomCol: "42000", - ErDataOutOfRange: "22003", - ErAccessDeniedNoPasswordError: "28000", - ErTruncateIllegalFk: "42000", - ErDaInvalidConditionNumber: "35000", - ErForeignDuplicateKeyWithChildInfo: "23000", - ErForeignDuplicateKeyWithoutChildInfo: "23000", - ErCantExecuteInReadOnlyTransaction: "25006", - ErAlterOperationNotSupported: "0A000", - ErAlterOperationNotSupportedReason: "0A000", - ErDupUnknownInIndex: "23000", + ErrDupKey: "23000", + ErrOutofmemory: "HY001", + ErrOutOfSortmemory: "HY001", + ErrConCount: "08004", + ErrBadHost: "08S01", + ErrHandshake: "08S01", + ErrDbaccessDenied: "42000", + ErrAccessDenied: "28000", + ErrNoDb: "3D000", + ErrUnknownCom: "08S01", + ErrBadNull: "23000", + ErrBadDb: "42000", + ErrTableExists: "42S01", + ErrBadTable: "42S02", + ErrNonUniq: "23000", + ErrServerShutdown: "08S01", + ErrBadField: "42S22", + ErrWrongFieldWithGroup: "42000", + ErrWrongSumSelect: "42000", + ErrWrongGroupField: "42000", + ErrWrongValueCount: "21S01", + ErrTooLongIdent: "42000", + ErrDupFieldname: "42S21", + ErrDupKeyname: "42000", + ErrDupEntry: "23000", + ErrWrongFieldSpec: "42000", + ErrParse: "42000", + ErrEmptyQuery: "42000", + ErrNonuniqTable: "42000", + ErrInvalidDefault: "42000", + ErrMultiplePriKey: "42000", + ErrTooManyKeys: "42000", + ErrTooManyKeyParts: "42000", + ErrTooLongKey: "42000", + ErrKeyColumnDoesNotExits: "42000", + ErrBlobUsedAsKey: "42000", + ErrTooBigFieldlength: "42000", + ErrWrongAutoKey: "42000", + ErrForcingClose: "08S01", + ErrIpsock: "08S01", + ErrNoSuchIndex: "42S12", + ErrWrongFieldTerminators: "42000", + ErrBlobsAndNoTerminated: "42000", + ErrCantRemoveAllFields: "42000", + ErrCantDropFieldOrKey: "42000", + ErrBlobCantHaveDefault: "42000", + ErrWrongDbName: "42000", + ErrWrongTableName: "42000", + ErrTooBigSelect: "42000", + ErrUnknownProcedure: "42000", + ErrWrongParamcountToProcedure: "42000", + ErrUnknownTable: "42S02", + ErrFieldSpecifiedTwice: "42000", + ErrUnsupportedExtension: "42000", + ErrTableMustHaveColumns: "42000", + ErrUnknownCharacterSet: "42000", + ErrTooBigRowsize: "42000", + ErrWrongOuterJoin: "42000", + ErrNullColumnInIndex: "42000", + ErrPasswordAnonymousUser: "42000", + ErrPasswordNotAllowed: "42000", + ErrPasswordNoMatch: "42000", + ErrWrongValueCountOnRow: "21S01", + ErrInvalidUseOfNull: "22004", + ErrRegexp: "42000", + ErrMixOfGroupFuncAndFields: "42000", + ErrNonexistingGrant: "42000", + ErrTableaccessDenied: "42000", + ErrColumnaccessDenied: "42000", + ErrIllegalGrantForTable: "42000", + ErrGrantWrongHostOrUser: "42000", + ErrNoSuchTable: "42S02", + ErrNonexistingTableGrant: "42000", + ErrNotAllowedCommand: "42000", + ErrSyntax: "42000", + ErrAbortingConnection: "08S01", + ErrNetPacketTooLarge: "08S01", + ErrNetReadErrorFromPipe: "08S01", + ErrNetFcntl: "08S01", + ErrNetPacketsOutOfOrder: "08S01", + ErrNetUncompress: "08S01", + ErrNetRead: "08S01", + ErrNetReadInterrupted: "08S01", + ErrNetErrorOnWrite: "08S01", + ErrNetWriteInterrupted: "08S01", + ErrTooLongString: "42000", + ErrTableCantHandleBlob: "42000", + ErrTableCantHandleAutoIncrement: "42000", + ErrWrongColumnName: "42000", + ErrWrongKeyColumn: "42000", + ErrDupUnique: "23000", + ErrBlobKeyWithoutLength: "42000", + ErrPrimaryCantHaveNull: "42000", + ErrTooManyRows: "42000", + ErrRequiresPrimaryKey: "42000", + ErrKeyDoesNotExits: "42000", + ErrCheckNoSuchTable: "42000", + ErrCheckNotImplemented: "42000", + ErrCantDoThisDuringAnTransaction: "25000", + ErrNewAbortingConnection: "08S01", + ErrMasterNetRead: "08S01", + ErrMasterNetWrite: "08S01", + ErrTooManyUserConnections: "42000", + ErrReadOnlyTransaction: "25000", + ErrNoPermissionToCreateUser: "42000", + ErrLockDeadlock: "40001", + ErrNoReferencedRow: "23000", + ErrRowIsReferenced: "23000", + ErrConnectToMaster: "08S01", + ErrWrongNumberOfColumnsInSelect: "21000", + ErrUserLimitReached: "42000", + ErrSpecificAccessDenied: "42000", + ErrNoDefault: "42000", + ErrWrongValueForVar: "42000", + ErrWrongTypeForVar: "42000", + ErrCantUseOptionHere: "42000", + ErrNotSupportedYet: "42000", + ErrWrongFkDef: "42000", + ErrOperandColumns: "21000", + ErrSubqueryNo1Row: "21000", + ErrIllegalReference: "42S22", + ErrDerivedMustHaveAlias: "42000", + ErrSelectReduced: "01000", + ErrTablenameNotAllowedHere: "42000", + ErrNotSupportedAuthMode: "08004", + ErrSpatialCantHaveNull: "42000", + ErrCollationCharsetMismatch: "42000", + ErrWarnTooFewRecords: "01000", + ErrWarnTooManyRecords: "01000", + ErrWarnNullToNotnull: "22004", + ErrWarnDataOutOfRange: "22003", + WarnDataTruncated: "01000", + ErrWrongNameForIndex: "42000", + ErrWrongNameForCatalog: "42000", + ErrUnknownStorageEngine: "42000", + ErrTruncatedWrongValue: "22007", + ErrSpNoRecursiveCreate: "2F003", + ErrSpAlreadyExists: "42000", + ErrSpDoesNotExist: "42000", + ErrSpLilabelMismatch: "42000", + ErrSpLabelRedefine: "42000", + ErrSpLabelMismatch: "42000", + ErrSpUninitVar: "01000", + ErrSpBadselect: "0A000", + ErrSpBadreturn: "42000", + ErrSpBadstatement: "0A000", + ErrUpdateLogDeprecatedIgnored: "42000", + ErrUpdateLogDeprecatedTranslated: "42000", + ErrQueryInterrupted: "70100", + ErrSpWrongNoOfArgs: "42000", + ErrSpCondMismatch: "42000", + ErrSpNoreturn: "42000", + ErrSpNoreturnend: "2F005", + ErrSpBadCursorQuery: "42000", + ErrSpBadCursorSelect: "42000", + ErrSpCursorMismatch: "42000", + ErrSpCursorAlreadyOpen: "24000", + ErrSpCursorNotOpen: "24000", + ErrSpUndeclaredVar: "42000", + ErrSpFetchNoData: "02000", + ErrSpDupParam: "42000", + ErrSpDupVar: "42000", + ErrSpDupCond: "42000", + ErrSpDupCurs: "42000", + ErrSpSubselectNyi: "0A000", + ErrStmtNotAllowedInSfOrTrg: "0A000", + ErrSpVarcondAfterCurshndlr: "42000", + ErrSpCursorAfterHandler: "42000", + ErrSpCaseNotFound: "20000", + ErrDivisionByZero: "22012", + ErrIllegalValueForType: "22007", + ErrProcaccessDenied: "42000", + ErrXaerNota: "XAE04", + ErrXaerInval: "XAE05", + ErrXaerRmfail: "XAE07", + ErrXaerOutside: "XAE09", + ErrXaerRmerr: "XAE03", + ErrXaRbrollback: "XA100", + ErrNonexistingProcGrant: "42000", + ErrDataTooLong: "22001", + ErrSpBadSQLstate: "42000", + ErrCantCreateUserWithGrant: "42000", + ErrSpDupHandler: "42000", + ErrSpNotVarArg: "42000", + ErrSpNoRetset: "0A000", + ErrCantCreateGeometryObject: "22003", + ErrTooBigScale: "42000", + ErrTooBigPrecision: "42000", + ErrMBiggerThanD: "42000", + ErrTooLongBody: "42000", + ErrTooBigDisplaywidth: "42000", + ErrXaerDupid: "XAE08", + ErrDatetimeFunctionOverflow: "22008", + ErrRowIsReferenced2: "23000", + ErrNoReferencedRow2: "23000", + ErrSpBadVarShadow: "42000", + ErrSpWrongName: "42000", + ErrSpNoAggregate: "42000", + ErrMaxPreparedStmtCountReached: "42000", + ErrNonGroupingFieldUsed: "42000", + ErrForeignDuplicateKeyOldUnused: "23000", + ErrCantChangeTxCharacteristics: "25001", + ErrWrongParamcountToNativeFct: "42000", + ErrWrongParametersToNativeFct: "42000", + ErrWrongParametersToStoredFct: "42000", + ErrDupEntryWithKeyName: "23000", + ErrXaRbtimeout: "XA106", + ErrXaRbdeadlock: "XA102", + ErrFuncInexistentNameCollision: "42000", + ErrDupSignalSet: "42000", + ErrSignalWarn: "01000", + ErrSignalNotFound: "02000", + ErrSignalException: "HY000", + ErrResignalWithoutActiveHandler: "0K000", + ErrSpatialMustHaveGeomCol: "42000", + ErrDataOutOfRange: "22003", + ErrAccessDeniedNoPassword: "28000", + ErrTruncateIllegalFk: "42000", + ErrDaInvalidConditionNumber: "35000", + ErrForeignDuplicateKeyWithChildInfo: "23000", + ErrForeignDuplicateKeyWithoutChildInfo: "23000", + ErrCantExecuteInReadOnlyTransaction: "25006", + ErrAlterOperationNotSupported: "0A000", + ErrAlterOperationNotSupportedReason: "0A000", + ErrDupUnknownInIndex: "23000", } diff --git a/mysqldef/util.go b/mysqldef/util.go index da8c9c7927..11f08c03a3 100644 --- a/mysqldef/util.go +++ b/mysqldef/util.go @@ -17,6 +17,7 @@ package mysqldef // Call this when no Flen assigned in ddl. // or column value is calculated from an expression. // For example: "select count(*) from t;", the column type is int64 and Flen in ResultField will be 21. +// See: https://dev.mysql.com/doc/refman/5.7/en/storage-requirements.html func GetDefaultFieldLength(tp byte) int { switch tp { case TypeTiny: @@ -29,8 +30,25 @@ func GetDefaultFieldLength(tp byte) int { return 11 case TypeLonglong: return 21 + case TypeDecimal: + // See: https://dev.mysql.com/doc/refman/5.7/en/fixed-point-types.html + return 10 + case TypeBit, TypeBlob: + return -1 default: //TODO: add more types - return 0 + return -1 + } +} + +// GetDefaultDecimal returns the default decimal length for column. +func GetDefaultDecimal(tp byte) int { + switch tp { + case TypeDecimal: + // See: https://dev.mysql.com/doc/refman/5.7/en/fixed-point-types.html + return 0 + default: + //TODO: add more types + return -1 } } diff --git a/mysqldef/util_test.go b/mysqldef/util_test.go index 1baff0c452..e2e98a9f9d 100644 --- a/mysqldef/util_test.go +++ b/mysqldef/util_test.go @@ -25,7 +25,9 @@ func TestGetFieldLength(t *testing.T) { {TypeInt24, 9}, {TypeLong, 11}, {TypeLonglong, 21}, - {TypeNull, 0}, + {TypeBit, -1}, + {TypeBlob, -1}, + {TypeNull, -1}, } for _, test := range tbl { diff --git a/optimizer/optimizer.go b/optimizer/optimizer.go new file mode 100644 index 0000000000..36ccd06c5f --- /dev/null +++ b/optimizer/optimizer.go @@ -0,0 +1,32 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package optimizer + +import ( + "github.com/pingcap/tidb/ast" + "github.com/pingcap/tidb/stmt" +) + +// Compile compiles a ast.Node into a executable statement. +func Compile(node ast.Node) (stmt.Statement, error) { + switch v := node.(type) { + case *ast.SetStmt: + return compileSet(v) + } + return nil, nil +} + +func compileSet(aset *ast.SetStmt) (stmt.Statement, error) { + return nil, nil +} diff --git a/parser/coldef/col_def.go b/parser/coldef/col_def.go index 041afb6832..8010c08dec 100644 --- a/parser/coldef/col_def.go +++ b/parser/coldef/col_def.go @@ -181,9 +181,12 @@ func ColumnDefToCol(offset int, colDef *ColumnDef) (*column.Col, []*TableConstra } // If flen is not assigned, assigned it by type. - if col.Flen == 0 { + if col.Flen == types.UnspecifiedLength { col.Flen = mysql.GetDefaultFieldLength(col.Tp) } + if col.Decimal == types.UnspecifiedLength { + col.Decimal = mysql.GetDefaultDecimal(col.Tp) + } setOnUpdateNow := false hasDefaultValue := false diff --git a/parser/parser.y b/parser/parser.y index 0f9801ffca..c782bbb6c5 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -30,6 +30,7 @@ import ( "strings" mysql "github.com/pingcap/tidb/mysqldef" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/parser/coldef" "github.com/pingcap/tidb/ddl" "github.com/pingcap/tidb/expression" @@ -2931,15 +2932,29 @@ RollbackStmt: } SelectStmt: - "SELECT" SelectStmtOpts SelectStmtFieldList FromDual SelectStmtLimit SelectLockOpt + "SELECT" SelectStmtOpts SelectStmtFieldList SelectStmtLimit SelectLockOpt { $$ = &stmts.SelectStmt { Distinct: $2.(bool), Fields: $3.([]*field.Field), - From: nil, - Lock: $6.(coldef.LockType), + Lock: $5.(coldef.LockType), } } +| "SELECT" SelectStmtOpts SelectStmtFieldList FromDual WhereClauseOptional SelectStmtLimit SelectLockOpt + { + st := &stmts.SelectStmt { + Distinct: $2.(bool), + Fields: $3.([]*field.Field), + From: nil, + Lock: $7.(coldef.LockType), + } + + if $5 != nil { + st.Where = &rsets.WhereRset{Expr: $5.(expression.Expression)} + } + + $$ = st + } | "SELECT" SelectStmtOpts SelectStmtFieldList "FROM" FromClause WhereClauseOptional SelectStmtGroup HavingClause SelectStmtOrder SelectStmtLimit SelectLockOpt @@ -2977,8 +2992,7 @@ SelectStmt: } FromDual: - /* Empty */ -| "FROM" "DUAL" + "FROM" "DUAL" FromClause: @@ -3523,9 +3537,15 @@ StatementList: Statement { if $1 != nil { - s := $1.(stmt.Statement) - s.SetText(yylex.(*lexer).stmtText()) - yylex.(*lexer).list = []stmt.Statement{ s } + n, ok := $1.(ast.Node) + if ok { + n.SetText(yylex.(*lexer).stmtText()) + yylex.(*lexer).list = []interface{}{n} + } else { + s := $1.(stmt.Statement) + s.SetText(yylex.(*lexer).stmtText()) + yylex.(*lexer).list = []interface{}{s} + } } } | StatementList ';' Statement @@ -3861,21 +3881,25 @@ BitValueType: } StringType: - NationalOpt "CHAR" FieldLen OptBinary + NationalOpt "CHAR" FieldLen OptBinary OptCharset OptCollate { x := types.NewFieldType(mysql.TypeString) x.Flen = $3.(int) if $4.(bool) { x.Flag |= mysql.BinaryFlag } + x.Charset = $5.(string) + x.Collate = $6.(string) $$ = x } -| NationalOpt "CHAR" OptBinary +| NationalOpt "CHAR" OptBinary OptCharset OptCollate { x := types.NewFieldType(mysql.TypeString) if $3.(bool) { x.Flag |= mysql.BinaryFlag } + x.Charset = $4.(string) + x.Collate = $5.(string) $$ = x } | NationalOpt "VARCHAR" FieldLen OptBinary OptCharset OptCollate diff --git a/parser/parser_test.go b/parser/parser_test.go index 12677906ef..432f87dc54 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -278,6 +278,8 @@ func (s *testParserSuite) TestDMLStmt(c *C) { // For dual {"select 1 from dual", true}, {"select 1 from dual limit 1", true}, + {"select 1 where exists (select 2)", false}, + {"select 1 from dual where not exists (select 2)", true}, // For show create table {"show create table test.t", true}, @@ -442,6 +444,11 @@ func (s *testParserSuite) TestDDL(c *C) { {"CREATE TABLE foo (a SMALLINT UNSIGNED, b INT UNSIGNED) // foo", true}, {"CREATE TABLE foo (a SMALLINT UNSIGNED, b INT UNSIGNED) /* foo */", true}, {"CREATE TABLE foo /* foo */ (a SMALLINT UNSIGNED, b INT UNSIGNED) /* foo */", true}, + {"CREATE TABLE foo (name CHAR(50) BINARY)", true}, + {"CREATE TABLE foo (name CHAR(50) COLLATE utf8_bin)", true}, + {"CREATE TABLE foo (name CHAR(50) CHARACTER SET utf8)", true}, + {"CREATE TABLE foo (name CHAR(50) BINARY CHARACTER SET utf8 COLLATE utf8_bin)", true}, + {"CREATE TABLE foo (a.b, b);", false}, {"CREATE TABLE foo (a, b.c);", false}, // For table option diff --git a/parser/scanner.l b/parser/scanner.l index b84c6f4cfb..0364493921 100644 --- a/parser/scanner.l +++ b/parser/scanner.l @@ -26,7 +26,6 @@ import ( "strings" "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/stmt" mysql "github.com/pingcap/tidb/mysqldef" ) @@ -39,7 +38,7 @@ type lexer struct { inj int lcol int line int - list []stmt.Statement + list []interface{} ncol int nline int sc int @@ -73,7 +72,7 @@ func (l *lexer) Errors() []error { return l.errs } -func (l *lexer) Stmts() []stmt.Statement{ +func (l *lexer) Stmts() []interface{}{ return l.list } diff --git a/plan/plans/having.go b/plan/plans/having.go index 82a50c0860..231f059eaa 100644 --- a/plan/plans/having.go +++ b/plan/plans/having.go @@ -32,6 +32,7 @@ type HavingPlan struct { Src plan.Plan Expr expression.Expression evalArgs map[interface{}]interface{} + *SelectList } // Explain implements plan.Plan Explain interface. diff --git a/plan/plans/having_test.go b/plan/plans/having_test.go index c54ea143cc..a3777fe9f2 100644 --- a/plan/plans/having_test.go +++ b/plan/plans/having_test.go @@ -30,6 +30,10 @@ var _ = Suite(&testHavingPlan{}) func (t *testHavingPlan) TestHaving(c *C) { tblPlan := &testTablePlan{groupByTestData, []string{"id", "name"}, 0} havingPlan := &plans.HavingPlan{ + SelectList: &plans.SelectList{ + HiddenFieldOffset: len(tblPlan.GetFields()), + ResultFields: tblPlan.GetFields(), + }, Src: tblPlan, Expr: &expression.BinaryOperation{ Op: opcode.GE, diff --git a/plan/plans/index.go b/plan/plans/index.go index a91af7f8f0..fcba257e7f 100644 --- a/plan/plans/index.go +++ b/plan/plans/index.go @@ -194,6 +194,7 @@ func (r *indexPlan) GetFields() []*field.ResultField { // Filter implements plan.Plan Filter interface. // Filter merges BinaryOperations and determines the lower and upper bound. func (r *indexPlan) Filter(ctx context.Context, expr expression.Expression) (plan.Plan, bool, error) { + var spans []*indexSpan switch x := expr.(type) { case *expression.BinaryOperation: ok, name, val, err := x.IsIdentCompareVal() @@ -214,14 +215,12 @@ func (r *indexPlan) Filter(ctx context.Context, expr expression.Expression) (pla if err != nil { return nil, false, errors.Trace(err) } - r.spans = filterSpans(r.spans, toSpans(x.Op, val, seekVal)) - return r, true, nil + spans = filterSpans(r.spans, toSpans(x.Op, val, seekVal)) case *expression.Ident: if r.col.Name.L != x.L { break } - r.spans = filterSpans(r.spans, toSpans(opcode.GE, minNotNullVal, nil)) - return r, true, nil + spans = filterSpans(r.spans, toSpans(opcode.GE, minNotNullVal, nil)) case *expression.UnaryOperation: if x.Op != '!' { break @@ -234,16 +233,25 @@ func (r *indexPlan) Filter(ctx context.Context, expr expression.Expression) (pla if r.col.Name.L != cname.L { break } - r.spans = filterSpans(r.spans, toSpans(opcode.EQ, nil, nil)) - return r, true, nil + spans = filterSpans(r.spans, toSpans(opcode.EQ, nil, nil)) } - return r, false, nil + if spans == nil { + return r, false, nil + } + + return &indexPlan{ + src: r.src, + col: r.col, + idxName: r.idxName, + idx: r.idx, + spans: spans, + }, true, nil } // return the intersection range between origin and filter. func filterSpans(origin []*indexSpan, filter []*indexSpan) []*indexSpan { - var newSpans []*indexSpan + newSpans := make([]*indexSpan, 0, len(filter)) for _, fSpan := range filter { for _, oSpan := range origin { newSpan := oSpan.cutOffLow(fSpan.lowVal, fSpan.lowExclude) diff --git a/plan/plans/info.go b/plan/plans/info.go index 2779dfbe1e..9b222c7a6a 100644 --- a/plan/plans/info.go +++ b/plan/plans/info.go @@ -361,22 +361,21 @@ func (isp *InfoSchemaPlan) fetchColumns(schemas []*model.DBInfo) { if decimal == types.UnspecifiedLength { decimal = 0 } - dataType := types.TypeToStr(col.Tp, col.Charset == charset.CharsetBin) - columnType := fmt.Sprintf("%s(%d)", dataType, colLen) + columnType := col.FieldType.CompactStr() columnDesc := column.NewColDesc(&column.Col{ColumnInfo: *col}) var columnDefault interface{} if columnDesc.DefaultValue != nil { columnDefault = fmt.Sprintf("%v", columnDesc.DefaultValue) } record := []interface{}{ - catalogVal, // TABLE_CATALOG - schema.Name.O, // TABLE_SCHEMA - table.Name.O, // TABLE_NAME - col.Name.O, // COLUMN_NAME - i + 1, // ORIGINAL_POSITION - columnDefault, // COLUMN_DEFAULT - columnDesc.Null, // IS_NULLABLE - types.TypeToStr(col.Tp, col.Charset == charset.CharsetBin), // DATA_TYPE + catalogVal, // TABLE_CATALOG + schema.Name.O, // TABLE_SCHEMA + table.Name.O, // TABLE_NAME + col.Name.O, // COLUMN_NAME + i + 1, // ORIGINAL_POSITION + columnDefault, // COLUMN_DEFAULT + columnDesc.Null, // IS_NULLABLE + types.TypeToStr(col.Tp, col.Charset), // DATA_TYPE colLen, // CHARACTER_MAXIMUM_LENGTH colLen, // CHARACTOR_OCTET_LENGTH decimal, // NUMERIC_PRECISION diff --git a/plan/plans/info_test.go b/plan/plans/info_test.go index c39898f2ec..14af54a4b2 100644 --- a/plan/plans/info_test.go +++ b/plan/plans/info_test.go @@ -62,6 +62,9 @@ func mustQuery(c *C, currDB *sql.DB, s string) int { c.Assert(err, IsNil) cnt++ } + c.Assert(r.Err(), IsNil) + r.Close() + mustCommit(c, tx) return cnt } diff --git a/plan/plans/join.go b/plan/plans/join.go index 2c9e7c9ad9..cd04caa6cd 100644 --- a/plan/plans/join.go +++ b/plan/plans/join.go @@ -289,6 +289,7 @@ func (r *JoinPlan) nextCrossJoin(ctx context.Context) (row *plan.Row, err error) if r.curRow == nil { return nil, nil } + if r.On != nil { tempExpr := r.On.Clone() visitor := NewIdentEvalVisitor(r.Left.GetFields(), r.curRow.Data) @@ -316,7 +317,11 @@ func (r *JoinPlan) nextCrossJoin(ctx context.Context) (row *plan.Row, err error) r.tempPlan.Close() continue } - joinedRow := append(r.curRow.Data, rightRow.Data...) + + // To prevent outer modify the slice. See comment above. + joinedRow := make([]interface{}, 0, len(r.curRow.Data)+len(rightRow.Data)) + joinedRow = append(append(joinedRow, r.curRow.Data...), rightRow.Data...) + if r.On != nil { r.evalArgs[expression.ExprEvalIdentFunc] = func(name string) (interface{}, error) { return GetIdentValue(name, r.Fields, joinedRow, field.DefaultFieldFlag) diff --git a/plan/plans/select_list.go b/plan/plans/select_list.go index d98abd47b4..7d918fceed 100644 --- a/plan/plans/select_list.go +++ b/plan/plans/select_list.go @@ -59,10 +59,13 @@ func (s *SelectList) updateFields(table string, resultFields []*field.ResultFiel func createEmptyResultField(f *field.Field) *field.ResultField { result := &field.ResultField{} + // Set origin name + result.ColumnInfo.Name = model.NewCIStr(f.Expr.String()) + if len(f.AsName) > 0 { result.Name = f.AsName } else { - result.Name = f.Expr.String() + result.Name = result.ColumnInfo.Name.O } return result } @@ -111,7 +114,8 @@ func (s *SelectList) UpdateAggFields(expr expression.Expression, tableFields []* // We must add aggregate function to hidden select list // and use a position expression to fetch its value later. exprName := expr.String() - if !field.ContainFieldName(exprName, s.ResultFields, field.CheckFieldFlag) { + idx := field.GetResultFieldIndex(exprName, s.ResultFields, field.CheckFieldFlag) + if len(idx) == 0 { f := &field.Field{Expr: expr} resultField := &field.ResultField{Name: exprName} s.AddField(f, resultField) @@ -119,7 +123,8 @@ func (s *SelectList) UpdateAggFields(expr expression.Expression, tableFields []* return &expression.Position{N: len(s.Fields), Name: exprName}, nil } - return nil, nil + // select list has this field, use it directly. + return &expression.Position{N: idx[0] + 1, Name: exprName}, nil } // CloneHiddenField checks and clones field and result field from table fields, @@ -140,6 +145,55 @@ func (s *SelectList) CloneHiddenField(name string, tableFields []*field.ResultFi return false } +// CheckReferAmbiguous checks whether an identifier reference is ambiguous or not in select list. +// e,g, "select c1 as a, c2 as a from t group by a" is ambiguous, +// but "select c1 as a, c1 as a from t group by a" is not. +// For MySQL "select c1 as a, c2 + 1 as a from t group by a" is not ambiguous too, +// so we will only check identifier too. +// If no ambiguous, -1 means expr refers none in select list, else an index in select list returns. +func (s *SelectList) CheckReferAmbiguous(expr expression.Expression) (int, error) { + if _, ok := expr.(*expression.Ident); !ok { + return -1, nil + } + + name := expr.String() + if field.IsQualifiedName(name) { + // name is qualified, no need to check + return -1, nil + } + + lastIndex := -1 + // only check origin select list, no hidden field. + for i := 0; i < s.HiddenFieldOffset; i++ { + if !strings.EqualFold(s.ResultFields[i].Name, name) { + continue + } else if _, ok := s.Fields[i].Expr.(*expression.Ident); !ok { + // not identfier, no check + continue + } + + if lastIndex == -1 { + // first match, continue + lastIndex = i + continue + } + + // check origin name, e,g. "select c1 as c2, c2 from t group by c2" is ambiguous. + if s.ResultFields[i].ColumnInfo.Name.L != s.ResultFields[lastIndex].ColumnInfo.Name.L { + return -1, errors.Errorf("refer %s is ambiguous", expr) + } + + // check table name, e.g, "select t.c1, c1 from t group by c1" is not ambiguous. + if s.ResultFields[i].TableName != s.ResultFields[lastIndex].TableName { + return -1, errors.Errorf("refer %s is ambiguous", expr) + } + + // TODO: check database name if possible. + } + + return lastIndex, nil +} + // ResolveSelectList gets fields and result fields from selectFields and srcFields, // including field validity check and wildcard field processing. func ResolveSelectList(selectFields []*field.Field, srcFields []*field.ResultField) (*SelectList, error) { @@ -180,8 +234,16 @@ func ResolveSelectList(selectFields []*field.Field, srcFields []*field.ResultFie } var result *field.ResultField - if err = field.CheckAllFieldNames(names, srcFields, field.DefaultFieldFlag); err != nil { - return nil, errors.Trace(err) + for _, name := range names { + idx := field.GetResultFieldIndex(name, srcFields, field.DefaultFieldFlag) + if len(idx) > 1 { + return nil, errors.Errorf("ambiguous field %s", name) + } + + // TODO: must check in outer query too. + if len(idx) == 0 { + return nil, errors.Errorf("unknown field %s", name) + } } if _, ok := v.Expr.(*expression.Ident); ok { diff --git a/plan/plans/select_list_test.go b/plan/plans/select_list_test.go new file mode 100644 index 0000000000..383a14e69b --- /dev/null +++ b/plan/plans/select_list_test.go @@ -0,0 +1,111 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package plans_test + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/field" + "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/plan/plans" +) + +type testSelectListSuite struct{} + +var _ = Suite(&testSelectListSuite{}) + +func (s *testSelectListSuite) createTestResultFields(colNames []string) []*field.ResultField { + fs := make([]*field.ResultField, 0, len(colNames)) + + for i := 0; i < len(colNames); i++ { + f := &field.ResultField{ + TableName: "t", + Name: colNames[i], + } + + f.ColumnInfo.Name = model.NewCIStr(colNames[i]) + + fs = append(fs, f) + } + + return fs +} + +func (s *testSelectListSuite) TestAmbiguous(c *C) { + type pair struct { + Name string + AsName string + } + + createResultFields := func(names []string) []*field.ResultField { + fs := make([]*field.ResultField, 0, len(names)) + + for i := 0; i < len(names); i++ { + f := &field.ResultField{ + TableName: "t", + Name: names[i], + } + + f.ColumnInfo.Name = model.NewCIStr(names[i]) + + fs = append(fs, f) + } + + return fs + } + + createFields := func(ps []pair) []*field.Field { + fields := make([]*field.Field, len(ps)) + for i, f := range ps { + fields[i] = &field.Field{ + Expr: &expression.Ident{ + CIStr: model.NewCIStr(f.Name), + }, + AsName: f.AsName, + } + } + return fields + } + + tbl := []struct { + Fields []pair + Name string + Err bool + Index int + }{ + {[]pair{{"id", ""}}, "id", false, 0}, + {[]pair{{"id", "a"}, {"name", "a"}}, "a", true, -1}, + {[]pair{{"id", "a"}, {"name", "a"}}, "id", false, -1}, + } + + for _, t := range tbl { + rs := createResultFields([]string{"id", "name"}) + fs := createFields(t.Fields) + + sl, err := plans.ResolveSelectList(fs, rs) + c.Assert(err, IsNil) + + index, err := sl.CheckReferAmbiguous(&expression.Ident{ + CIStr: model.NewCIStr(t.Name), + }) + + if t.Err { + c.Assert(err, NotNil) + continue + } + + c.Assert(err, IsNil) + c.Assert(t.Index, Equals, index) + } +} diff --git a/plan/plans/show.go b/plan/plans/show.go index a0d1e04464..87eff2e91a 100644 --- a/plan/plans/show.go +++ b/plan/plans/show.go @@ -132,7 +132,9 @@ func (s *ShowPlan) Filter(ctx context.Context, expr expression.Expression) (plan // Next implements plan.Plan Next interface. func (s *ShowPlan) Next(ctx context.Context) (row *plan.Row, err error) { if s.rows == nil { - s.fetchAll(ctx) + if err := s.fetchAll(ctx); err != nil { + return nil, errors.Trace(err) + } } if s.cursor == len(s.rows) { return @@ -192,12 +194,13 @@ func (s *ShowPlan) getTable(ctx context.Context) (table.Table, error) { is := sessionctx.GetDomain(ctx).InfoSchema() dbName := model.NewCIStr(s.DBName) if !is.SchemaExists(dbName) { - return nil, errors.Errorf("Can not find DB: %s", dbName) + // MySQL returns no such table here if database doesn't exist. + return nil, errors.Trace(mysql.NewErr(mysql.ErrNoSuchTable, s.DBName, s.TableName)) } tbName := model.NewCIStr(s.TableName) tb, err := is.TableByName(dbName, tbName) if err != nil { - return nil, errors.Errorf("Can not find table: %s", s.TableName) + return nil, errors.Trace(mysql.NewErr(mysql.ErrNoSuchTable, s.DBName, s.TableName)) } return tb, nil } diff --git a/plan/plans/show_test.go b/plan/plans/show_test.go index f2ea971c34..7b36ebf810 100644 --- a/plan/plans/show_test.go +++ b/plan/plans/show_test.go @@ -224,4 +224,9 @@ func (p *testShowSuit) TestShowTables(c *C) { c.Assert(cnt, Equals, 2) cnt = mustQuery(c, testDB, `show full tables where Table_type != 'VIEW';`) c.Assert(cnt, Equals, 3) + + mustQuery(c, testDB, `show create table tab00;`) + rows, _ := testDB.Query(`show create table abc;`) + rows.Next() + c.Assert(rows.Err(), NotNil) } diff --git a/rset/rsets/groupby.go b/rset/rsets/groupby.go index ceee498980..aa3560bf97 100644 --- a/rset/rsets/groupby.go +++ b/rset/rsets/groupby.go @@ -39,32 +39,9 @@ type GroupByRset struct { SelectList *plans.SelectList } -// HasAmbiguousField checks whether have ambiguous group by fields. -func (r *GroupByRset) HasAmbiguousField(indices []int, fields []*field.Field) bool { - columnNameMap := map[string]struct{}{} - for _, index := range indices { - expr := fields[index].Expr - - // `select c1 + c2 as c1, c1 + c3 as c1 from t order by c1` is valid, - // if it is not `Ident` expression, ignore it. - v, ok := expr.(*expression.Ident) - if !ok { - continue - } - - // `select c1 as c2, c1 as c2 from t order by c2` is valid, - // use a map for it here. - columnNameMap[v.L] = struct{}{} - } - - return len(columnNameMap) > 1 -} - // Plan gets GroupByDefaultPlan. func (r *GroupByRset) Plan(ctx context.Context) (plan.Plan, error) { fields := r.SelectList.Fields - resultfields := r.SelectList.ResultFields - srcFields := r.Src.GetFields() r.SelectList.AggFields = GetAggFields(fields) aggFields := r.SelectList.AggFields @@ -93,30 +70,25 @@ func (r *GroupByRset) Plan(ctx context.Context) (plan.Plan, error) { // use Position expression for the associated field. r.By[i] = &expression.Position{N: position} } else { + index, err := r.SelectList.CheckReferAmbiguous(e) + if err != nil { + return nil, errors.Errorf("Column '%s' in group statement is ambiguous", e) + } else if _, ok := aggFields[index]; ok { + return nil, errors.Errorf("Can't group on '%s'", e) + } + + // TODO: check more ambiguous case + // Group by ambiguous rule: + // select c1 as a, c2 as a from t group by a is ambiguous + // select c1 as a, c2 as a from t group by a + 1 is ambiguous + // select c1 as c2, c2 from t group by c2 is ambiguous + // select c1 as c2, c2 from t group by c2 + 1 is ambiguous + + // TODO: use visitor to check aggregate function names := expression.MentionedColumns(e) for _, name := range names { - if field.ContainFieldName(name, srcFields, field.DefaultFieldFlag) { - // check whether column is qualified, like `select t.c1 c1, t.c2 from t group by t.c1, t.c2` - // no need to check ambiguous field. - if expression.IsQualified(name) { - continue - } - - // check ambiguous fields, like `select c1 as c2, c2 from t group by c2`. - if err := field.CheckAmbiguousField(name, resultfields, field.DefaultFieldFlag); err == nil { - continue - } - } - - // check reference to group function name - indices := field.GetFieldIndex(name, fields[0:r.SelectList.HiddenFieldOffset], field.CheckFieldFlag) - if len(indices) > 1 { - // check ambiguous fields, like `select c1 as a, c2 as a from t group by a`, - // notice that `select c2 as c2, c2 as c2 from t group by c2;` is valid. - if r.HasAmbiguousField(indices, fields[0:r.SelectList.HiddenFieldOffset]) { - return nil, errors.Errorf("Column '%s' in group statement is ambiguous", name) - } - } else if len(indices) == 1 { + indices := field.GetFieldIndex(name, fields[0:r.SelectList.HiddenFieldOffset], field.DefaultFieldFlag) + if len(indices) == 1 { // check reference to aggregate function, like `select c1, count(c1) as b from t group by b + 1`. index := indices[0] if _, ok := aggFields[index]; ok { diff --git a/rset/rsets/groupby_test.go b/rset/rsets/groupby_test.go index 80b28fdbee..9a87172316 100644 --- a/rset/rsets/groupby_test.go +++ b/rset/rsets/groupby_test.go @@ -125,48 +125,6 @@ func (s *testGroupByRsetSuite) TestGroupByRsetPlan(c *C) { c.Assert(err, NotNil) } -func (s *testGroupByRsetSuite) TestGroupByHasAmbiguousField(c *C) { - fld := &field.Field{Expr: expression.Value{Val: 1}} - - // check `1` - fields := []*field.Field{fld} - indices := []int{0} - - ret := s.r.HasAmbiguousField(indices, fields) - c.Assert(ret, IsFalse) - - // check `c1 as c2, c1 as c2` - fld = &field.Field{Expr: &expression.Ident{CIStr: model.NewCIStr("c1")}, AsName: "c2"} - fields = []*field.Field{fld, fld} - indices = []int{0, 1} - - ret = s.r.HasAmbiguousField(indices, fields) - c.Assert(ret, IsFalse) - - // check `c1+c2 as c2, c1+c3 as c2` - exprx := expression.NewBinaryOperation(opcode.Plus, expression.Value{Val: "c1"}, - expression.Value{Val: "c2"}) - expry := expression.NewBinaryOperation(opcode.Plus, expression.Value{Val: "c1"}, - expression.Value{Val: "c3"}) - - fldx := &field.Field{Expr: exprx, AsName: "c2"} - fldy := &field.Field{Expr: expry, AsName: "c2"} - fields = []*field.Field{fldx, fldy} - indices = []int{0, 1} - - ret = s.r.HasAmbiguousField(indices, fields) - c.Assert(ret, IsFalse) - - // check `c1 as c2, c3 as c2` - fldx = &field.Field{Expr: &expression.Ident{CIStr: model.NewCIStr("c1")}, AsName: "c2"} - fldy = &field.Field{Expr: &expression.Ident{CIStr: model.NewCIStr("c3")}, AsName: "c2"} - fields = []*field.Field{fldx, fldy} - indices = []int{0, 1} - - ret = s.r.HasAmbiguousField(indices, fields) - c.Assert(ret, IsTrue) -} - func (s *testGroupByRsetSuite) TestGroupByRsetString(c *C) { str := s.r.String() c.Assert(len(str), Greater, 0) diff --git a/rset/rsets/having.go b/rset/rsets/having.go index 73bd81ecb7..39ba3cc561 100644 --- a/rset/rsets/having.go +++ b/rset/rsets/having.go @@ -28,8 +28,9 @@ var ( // HavingRset is record set for having fields. type HavingRset struct { - Src plan.Plan - Expr expression.Expression + Src plan.Plan + Expr expression.Expression + SelectList *plans.SelectList } // CheckAndUpdateSelectList checks having fields validity and set hidden fields to selectList. @@ -90,7 +91,7 @@ func (r *HavingRset) CheckAndUpdateSelectList(selectList *plans.SelectList, grou // Plan gets HavingPlan. func (r *HavingRset) Plan(ctx context.Context) (plan.Plan, error) { - return &plans.HavingPlan{Src: r.Src, Expr: r.Expr}, nil + return &plans.HavingPlan{Src: r.Src, Expr: r.Expr, SelectList: r.SelectList}, nil } // String implements fmt.Stringer interface. diff --git a/rset/rsets/having_test.go b/rset/rsets/having_test.go index 7e2149a6f8..91a13e3052 100644 --- a/rset/rsets/having_test.go +++ b/rset/rsets/having_test.go @@ -35,12 +35,7 @@ func (s *testHavingRsetSuite) SetUpSuite(c *C) { // expr `id > 1` expr := expression.NewBinaryOperation(opcode.GT, &expression.Ident{CIStr: model.NewCIStr("id")}, expression.Value{Val: 1}) - s.r = &HavingRset{Src: tblPlan, Expr: expr} -} - -func (s *testHavingRsetSuite) TestHavingRsetCheckAndUpdateSelectList(c *C) { - resultFields := s.r.Src.GetFields() - + resultFields := tblPlan.GetFields() fields := make([]*field.Field, len(resultFields)) for i, resultField := range resultFields { name := resultField.Name @@ -52,6 +47,13 @@ func (s *testHavingRsetSuite) TestHavingRsetCheckAndUpdateSelectList(c *C) { ResultFields: resultFields, Fields: fields, } + s.r = &HavingRset{Src: tblPlan, Expr: expr, SelectList: selectList} +} + +func (s *testHavingRsetSuite) TestHavingRsetCheckAndUpdateSelectList(c *C) { + resultFields := s.r.Src.GetFields() + + selectList := s.r.SelectList groupBy := []expression.Expression{} diff --git a/rset/rsets/orderby.go b/rset/rsets/orderby.go index 9a8ed3db13..e0b9e74834 100644 --- a/rset/rsets/orderby.go +++ b/rset/rsets/orderby.go @@ -76,16 +76,24 @@ func (r *OrderByRset) CheckAndUpdateSelectList(selectList *plans.SelectList, tab r.By[i].Expr = expr } else { + if _, err := selectList.CheckReferAmbiguous(v.Expr); err != nil { + return errors.Errorf("Column '%s' in order statement is ambiguous", v.Expr) + } + + // TODO: check more ambiguous case + // Order by ambiguous rule: + // select c1 as a, c2 as a from t order by a is ambiguous + // select c1 as a, c2 as a from t order by a + 1 is ambiguous + // select c1 as c2, c2 from t order by c2 is ambiguous + // select c1 as c2, c2 from t order by c2 + 1 is ambiguous + + // TODO: use vistor to refactor all and combine following plan check. names := expression.MentionedColumns(v.Expr) for _, name := range names { // try to find in select list // TODO: mysql has confused result for this, see #555. // now we use select list then order by, later we should make it easier. if field.ContainFieldName(name, selectList.ResultFields, field.CheckFieldFlag) { - // check ambiguous fields, like `select c1 as c2, c2 from t order by c2`. - if err := field.CheckAmbiguousField(name, selectList.ResultFields, field.DefaultFieldFlag); err != nil { - return errors.Errorf("Column '%s' in order statement is ambiguous", name) - } continue } @@ -140,9 +148,16 @@ func (r *OrderByRset) Plan(ctx context.Context) (plan.Plan, error) { r.By[i].Expr = &expression.Position{N: position} } } else { + // Don't check ambiguous here, only check field exists or not. + // TODO: use visitor to refactor. colNames := expression.MentionedColumns(e) - if err := field.CheckAllFieldNames(colNames, fields, field.CheckFieldFlag); err != nil { - return nil, errors.Trace(err) + for _, name := range colNames { + if idx := field.GetResultFieldIndex(name, r.SelectList.ResultFields, field.DefaultFieldFlag); len(idx) == 0 { + // find in from + if idx = field.GetResultFieldIndex(name, r.SelectList.FromFields, field.DefaultFieldFlag); len(idx) == 0 { + return nil, errors.Errorf("unknown field %s", name) + } + } } } diff --git a/rset/rsets/where.go b/rset/rsets/where.go index d9904cc856..30729792a9 100644 --- a/rset/rsets/where.go +++ b/rset/rsets/where.go @@ -18,6 +18,7 @@ package rsets import ( + "github.com/juju/errors" "github.com/ngaut/log" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" @@ -188,13 +189,18 @@ func (r *WhereRset) Plan(ctx context.Context) (plan.Plan, error) { return r.planStatic(ctx, expr) } + var ( + src = r.Src + err error + ) + switch x := expr.(type) { case *expression.BinaryOperation: - return r.planBinOp(ctx, x) + src, err = r.planBinOp(ctx, x) case *expression.Ident: - return r.planIdent(ctx, x) + src, err = r.planIdent(ctx, x) case *expression.IsNull: - return r.planIsNull(ctx, x) + src, err = r.planIsNull(ctx, x) case *expression.PatternIn: // TODO: optimize // TODO: show plan @@ -203,10 +209,22 @@ func (r *WhereRset) Plan(ctx context.Context) (plan.Plan, error) { case *expression.PatternRegexp: // TODO: optimize case *expression.UnaryOperation: - return r.planUnaryOp(ctx, x) + src, err = r.planUnaryOp(ctx, x) default: log.Warnf("%v not supported in where rset now", r.Expr) } - return &plans.FilterDefaultPlan{Plan: r.Src, Expr: expr}, nil + if err != nil { + return nil, errors.Trace(err) + } + + if _, ok := src.(*plans.FilterDefaultPlan); ok { + return src, nil + } + + // We must use a FilterDefaultPlan here to wrap filtered plan. + // Alghough we can check where condition using index plan, we still need + // to check again after the FROM phase if the FROM phase contains outer join. + // TODO: if FROM phase doesn't contain outer join, we can return filtered plan directly. + return &plans.FilterDefaultPlan{Plan: src, Expr: expr}, nil } diff --git a/session.go b/session.go index 5073e97bde..31c6a703a7 100644 --- a/session.go +++ b/session.go @@ -28,6 +28,7 @@ import ( "github.com/juju/errors" "github.com/ngaut/log" + "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/field" "github.com/pingcap/tidb/kv" mysql "github.com/pingcap/tidb/mysqldef" @@ -40,6 +41,7 @@ import ( "github.com/pingcap/tidb/stmt/stmts" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/errors2" + "github.com/pingcap/tidb/util/sqlexec" ) // Session context @@ -228,6 +230,34 @@ func (s *session) Retry() error { return nil } +// ExecRestrictedSQL implements SQLHelper interface. +// This is used for executing some restricted sql statements. +func (s *session) ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recordset, error) { + if ctx.Value(&sqlexec.RestrictedSQLExecutorKeyType{}) != nil { + // We do not support run this function concurrently. + // TODO: Maybe we should remove this restriction latter. + return nil, errors.New("Should not call ExecRestrictedSQL concurrently.") + } + statements, err := Compile(sql) + if err != nil { + log.Errorf("Compile %s with error: %v", sql, err) + return nil, errors.Trace(err) + } + if len(statements) != 1 { + log.Errorf("ExecRestrictedSQL only executes one statement. Too many/few statement in %s", sql) + return nil, errors.New("Wrong number of statement.") + } + st := statements[0] + // Check statement for some restriction + // For example only support DML on system meta table. + // TODO: Add more restrictions. + log.Infof("Executing %s [%s]", st, sql) + ctx.SetValue(&sqlexec.RestrictedSQLExecutorKeyType{}, true) + defer ctx.ClearValue(&sqlexec.RestrictedSQLExecutorKeyType{}) + rs, err := st.Exec(ctx) + return rs, errors.Trace(err) +} + func (s *session) Execute(sql string) ([]rset.Recordset, error) { statements, err := Compile(sql) if err != nil { diff --git a/sessionctx/sessionctx_test.go b/sessionctx/domainctx_test.go similarity index 100% rename from sessionctx/sessionctx_test.go rename to sessionctx/domainctx_test.go diff --git a/stmt/stmts/account_manage.go b/stmt/stmts/account_manage.go index 072673b1e8..c40e8ebc5f 100644 --- a/stmt/stmts/account_manage.go +++ b/stmt/stmts/account_manage.go @@ -18,21 +18,18 @@ package stmts import ( + "fmt" "strings" "github.com/juju/errors" "github.com/pingcap/tidb/context" - "github.com/pingcap/tidb/expression" - "github.com/pingcap/tidb/model" mysql "github.com/pingcap/tidb/mysqldef" "github.com/pingcap/tidb/parser/coldef" - "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/rset" - "github.com/pingcap/tidb/rset/rsets" "github.com/pingcap/tidb/stmt" - "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/format" + "github.com/pingcap/tidb/util/sqlexec" ) /************************************************************************************ @@ -73,39 +70,13 @@ func (s *CreateUserStmt) SetText(text string) { s.Text = text } -func composeUserTableFilter(name string, host string) expression.Expression { - nameMatch := expression.NewBinaryOperation(opcode.EQ, &expression.Ident{CIStr: model.NewCIStr("User")}, &expression.Value{Val: name}) - hostMatch := expression.NewBinaryOperation(opcode.EQ, &expression.Ident{CIStr: model.NewCIStr("Host")}, &expression.Value{Val: host}) - return expression.NewBinaryOperation(opcode.AndAnd, nameMatch, hostMatch) -} - -func composeUserTableRset() *rsets.JoinRset { - return &rsets.JoinRset{ - Left: &rsets.TableSource{ - Source: table.Ident{ - Name: model.NewCIStr(mysql.UserTable), - Schema: model.NewCIStr(mysql.SystemDB), - }, - }, - } -} - func (s *CreateUserStmt) userExists(ctx context.Context, name string, host string) (bool, error) { - r := composeUserTableRset() - p, err := r.Plan(ctx) + sql := fmt.Sprintf(`SELECT * FROM %s.%s WHERE User="%s" AND Host="%s";`, mysql.SystemDB, mysql.UserTable, name, host) + rs, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, sql) if err != nil { return false, errors.Trace(err) } - where := &rsets.WhereRset{ - Src: p, - Expr: composeUserTableFilter(name, host), - } - p, err = where.Plan(ctx) - if err != nil { - return false, errors.Trace(err) - } - defer p.Close() - row, err := p.Next(ctx) + row, err := rs.Next() if err != nil { return false, errors.Trace(err) } @@ -114,14 +85,7 @@ func (s *CreateUserStmt) userExists(ctx context.Context, name string, host strin // Exec implements the stmt.Statement Exec interface. func (s *CreateUserStmt) Exec(ctx context.Context) (rset.Recordset, error) { - st := &InsertIntoStmt{ - TableIdent: table.Ident{ - Name: model.NewCIStr(mysql.UserTable), - Schema: model.NewCIStr(mysql.SystemDB), - }, - ColNames: []string{"Host", "User", "Password"}, - } - values := make([][]expression.Expression, 0, len(s.Specs)) + users := make([]string, 0, len(s.Specs)) for _, spec := range s.Specs { strs := strings.Split(spec.User, "@") userName := strs[0] @@ -136,22 +100,20 @@ func (s *CreateUserStmt) Exec(ctx context.Context) (rset.Recordset, error) { } continue } - value := make([]expression.Expression, 0, 3) - value = append(value, expression.Value{Val: host}) - value = append(value, expression.Value{Val: userName}) + pwd := "" if spec.AuthOpt.ByAuthString { - value = append(value, expression.Value{Val: util.EncodePassword(spec.AuthOpt.AuthString)}) + pwd = util.EncodePassword(spec.AuthOpt.AuthString) } else { - // TODO: Maybe we should hash the string here? - value = append(value, expression.Value{Val: util.EncodePassword(spec.AuthOpt.HashString)}) + pwd = util.EncodePassword(spec.AuthOpt.HashString) } - values = append(values, value) + user := fmt.Sprintf(`("%s", "%s", "%s")`, host, userName, pwd) + users = append(users, user) } - if len(values) == 0 { + if len(users) == 0 { return nil, nil } - st.Lists = values - _, err := st.Exec(ctx) + sql := fmt.Sprintf(`INSERT INTO %s.%s (Host, User, Password) VALUES %s;`, mysql.SystemDB, mysql.UserTable, strings.Join(users, ", ")) + _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, sql) if err != nil { return nil, errors.Trace(err) } @@ -188,20 +150,13 @@ func (s *SetPwdStmt) SetText(text string) { } // Exec implements the stmt.Statement Exec interface. -func (s *SetPwdStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) { - // If len(s.User) == 0, use CURRENT_USER() +func (s *SetPwdStmt) Exec(ctx context.Context) (rset.Recordset, error) { + // TODO: If len(s.User) == 0, use CURRENT_USER() strs := strings.Split(s.User, "@") userName := strs[0] host := strs[1] // Update mysql.user - asgn := expression.Assignment{ - ColName: "Password", - Expr: expression.Value{Val: util.EncodePassword(s.Password)}, - } - st := &UpdateStmt{ - TableRefs: composeUserTableRset(), - List: []expression.Assignment{asgn}, - Where: composeUserTableFilter(userName, host), - } - return st.Exec(ctx) + sql := fmt.Sprintf(`UPDATE %s.%s SET password="%s" WHERE User="%s" AND Host="%s";`, mysql.SystemDB, mysql.UserTable, util.EncodePassword(s.Password), userName, host) + _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, sql) + return nil, errors.Trace(err) } diff --git a/stmt/stmts/delete.go b/stmt/stmts/delete.go index 1ab6c224b4..d8a9a91b87 100644 --- a/stmt/stmts/delete.go +++ b/stmt/stmts/delete.go @@ -127,23 +127,27 @@ func (s *DeleteStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) { defer p.Close() tblIDMap := make(map[int64]bool, len(s.TableIdents)) // Get table alias map. - tblAliasMap := make(map[string]string) + tblNames := make(map[string]string) if s.MultiTable { // Delete from multiple tables should consider table ident list. fs := p.GetFields() for _, f := range fs { if f.TableName != f.OrgTableName { - tblAliasMap[f.TableName] = f.OrgTableName + tblNames[f.TableName] = f.OrgTableName + } else { + tblNames[f.TableName] = f.TableName } } for _, t := range s.TableIdents { // Consider DBName. - oname, ok := tblAliasMap[t.Name.O] - if ok { - t.Name.O = oname - t.Name.L = strings.ToLower(oname) + oname, ok := tblNames[t.Name.O] + if !ok { + return nil, errors.Errorf("Unknown table '%s' in MULTI DELETE", t.Name.O) } + t.Name.O = oname + t.Name.L = strings.ToLower(oname) + var tbl table.Table tbl, err = getTable(ctx, t) if err != nil { diff --git a/stmt/stmts/delete_test.go b/stmt/stmts/delete_test.go index eea4522028..c56d3a568f 100644 --- a/stmt/stmts/delete_test.go +++ b/stmt/stmts/delete_test.go @@ -199,5 +199,8 @@ func (s *testStmtSuite) TestQualifedDelete(c *C) { r = mustExec(c, s.testDB, "delete a, b from t1 as a join t2 as b where a.c2 = b.c1") checkResult(c, r, 2, 0) + _, err = s.testDB.Exec("delete t1, t2 from t1 as a join t2 as b where a.c2 = b.c1") + c.Assert(err, NotNil) + mustExec(c, s.testDB, "drop table t1, t2") } diff --git a/stmt/stmts/insert.go b/stmt/stmts/insert.go index a5abc80bec..374349b975 100644 --- a/stmt/stmts/insert.go +++ b/stmt/stmts/insert.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/tidb/column" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/field" "github.com/pingcap/tidb/kv" mysql "github.com/pingcap/tidb/mysqldef" "github.com/pingcap/tidb/plan" @@ -232,6 +233,12 @@ func (s *InsertIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) } insertValueCount := len(s.Lists[0]) + toUpdateColumns, err0 := getOnDuplicateUpdateColumns(s.OnDuplicate, t) + if err0 != nil { + return nil, errors.Trace(err0) + } + + toUpdateArgs := map[interface{}]interface{}{} for i, list := range s.Lists { r := make([]interface{}, len(tableCols)) valueCount := len(list) @@ -294,15 +301,20 @@ func (s *InsertIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) // On duplicate key Update the duplicate row. // Evaluate the updated value. // TODO: report rows affected and last insert id. - toUpdateColumns, _, err := getUpdateColumns(t, s.OnDuplicate, false, nil) - if err != nil { - return nil, errors.Trace(err) - } data, err := t.Row(ctx, h) if err != nil { return nil, errors.Trace(err) } - err = updateRecord(ctx, h, data, t, toUpdateColumns, s.OnDuplicate, r, nil) + + toUpdateArgs[expression.ExprEvalValuesFunc] = func(name string) (interface{}, error) { + c, err1 := findColumnByName(t, name) + if err1 != nil { + return nil, errors.Trace(err1) + } + return r[c.Offset], nil + } + + err = updateRecord(ctx, h, data, t, toUpdateColumns, toUpdateArgs, 0, true) if err != nil { return nil, errors.Trace(err) } @@ -311,6 +323,19 @@ func (s *InsertIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) return nil, nil } +func getOnDuplicateUpdateColumns(assignList []expression.Assignment, t table.Table) (map[int]expression.Assignment, error) { + m := make(map[int]expression.Assignment, len(assignList)) + + for _, v := range assignList { + c, err := findColumnByName(t, field.JoinQualifiedName("", v.TableName, v.ColName)) + if err != nil { + return nil, errors.Trace(err) + } + m[c.Offset] = v + } + return m, nil +} + func (s *InsertIntoStmt) initDefaultValues(ctx context.Context, t table.Table, cols []*column.Col, row []interface{}, marked map[int]struct{}) error { var err error var defaultValueCols []*column.Col diff --git a/stmt/stmts/insert_test.go b/stmt/stmts/insert_test.go index 474ac58357..daf8ccd897 100644 --- a/stmt/stmts/insert_test.go +++ b/stmt/stmts/insert_test.go @@ -89,4 +89,10 @@ func (s *testStmtSuite) TestInsert(c *C) { insertSQL := `insert into insert_test (id, c2) values (1, 1) on duplicate key update c2=10;` mustExec(c, s.testDB, insertSQL) + + insertSQL = `insert into insert_test (id, c2) values (1, 1) on duplicate key update insert_test.c2=10;` + mustExec(c, s.testDB, insertSQL) + + _, err = s.testDB.Exec(`insert into insert_test (id, c2) values(1, 1) on duplicate key update t.c2 = 10`) + c.Assert(err, NotNil) } diff --git a/stmt/stmts/select.go b/stmt/stmts/select.go index 7e6b81aa31..8b889dc5a2 100644 --- a/stmt/stmts/select.go +++ b/stmt/stmts/select.go @@ -207,8 +207,9 @@ func (s *SelectStmt) Plan(ctx context.Context) (plan.Plan, error) { if s := s.Having; s != nil { if r, err = (&rsets.HavingRset{ - Src: r, - Expr: s.Expr}).Plan(ctx); err != nil { + Src: r, + Expr: s.Expr, + SelectList: selectList}).Plan(ctx); err != nil { return nil, err } } diff --git a/stmt/stmts/update.go b/stmt/stmts/update.go index ce0b698162..4ea5da59dc 100644 --- a/stmt/stmts/update.go +++ b/stmt/stmts/update.go @@ -18,7 +18,7 @@ package stmts import ( - "strings" + "fmt" "github.com/juju/errors" "github.com/ngaut/log" @@ -84,47 +84,45 @@ func (s *UpdateStmt) SetText(text string) { s.Text = text } -func getUpdateColumns(t table.Table, assignList []expression.Assignment, isMultipleTable bool, tblAliasMap map[string]string) ([]*column.Col, []expression.Assignment, error) { - // TODO: We should check the validate if assignList in somewhere else. Maybe in building plan. - // TODO: We should use field.GetFieldIndex to replace this function. - tcols := make([]*column.Col, 0, len(assignList)) - tAsgns := make([]expression.Assignment, 0, len(assignList)) - tname := t.TableName() - for _, asgn := range assignList { - if isMultipleTable { - if tblAliasMap != nil { - if alias, ok := tblAliasMap[asgn.TableName]; ok { - if !strings.EqualFold(tname.O, alias) { - continue - } - } - } else if !strings.EqualFold(tname.O, asgn.TableName) { - continue - } - } - col := column.FindCol(t.Cols(), asgn.ColName) - if col == nil { - if isMultipleTable { - continue - } - return nil, nil, errors.Errorf("UPDATE: unknown column %s", asgn.ColName) - } - tcols = append(tcols, col) - tAsgns = append(tAsgns, asgn) +func findColumnByName(t table.Table, name string) (*column.Col, error) { + _, tableName, colName := field.SplitQualifiedName(name) + if len(tableName) > 0 && tableName != t.TableName().O { + return nil, errors.Errorf("unknown field %s.%s", tableName, colName) } - return tcols, tAsgns, nil + + c := column.FindCol(t.Cols(), colName) + if c == nil { + return nil, errors.Errorf("unknown field %s", colName) + } + return c, nil } -func getInsertValue(name string, cols []*column.Col, row []interface{}) (interface{}, error) { - for i, col := range cols { - if col.Name.L == name { - return row[i], nil +func getUpdateColumns(assignList []expression.Assignment, fields []*field.ResultField) (map[int]expression.Assignment, error) { + m := make(map[int]expression.Assignment, len(assignList)) + + for _, v := range assignList { + + name := v.ColName + if len(v.TableName) > 0 { + name = fmt.Sprintf("%s.%s", v.TableName, v.ColName) } + // use result fields to check assign list, otherwise use origin table columns + idx := field.GetResultFieldIndex(name, fields, field.DefaultFieldFlag) + if n := len(idx); n > 1 { + return nil, errors.Errorf("ambiguous field %s", name) + } else if n == 0 { + return nil, errors.Errorf("unknown field %s", name) + } + + m[idx[0]] = v } - return nil, errors.Errorf("unknown field %s", name) + + return m, nil } -func updateRecord(ctx context.Context, h int64, data []interface{}, t table.Table, tcols []*column.Col, assignList []expression.Assignment, insertData []interface{}, args map[interface{}]interface{}) error { +func updateRecord(ctx context.Context, h int64, data []interface{}, t table.Table, + updateColumns map[int]expression.Assignment, m map[interface{}]interface{}, + offset int, onDuplicateUpdate bool) error { if err := t.LockRow(ctx, h, true); err != nil { return errors.Trace(err) } @@ -133,29 +131,27 @@ func updateRecord(ctx context.Context, h int64, data []interface{}, t table.Tabl touched := make([]bool, len(t.Cols())) copy(oldData, data) - // Generate new values - m := args - if m == nil { - m = make(map[interface{}]interface{}, len(t.Cols())) - // Set parameter for evaluating expression. - for _, col := range t.Cols() { - m[col.Name.L] = data[col.Offset] - } - } - if insertData != nil { - m[expression.ExprEvalValuesFunc] = func(name string) (interface{}, error) { - return getInsertValue(name, t.Cols(), insertData) - } - } + cols := t.Cols() - for i, asgn := range assignList { + assignExists := false + for i, asgn := range updateColumns { + if i < offset || i >= offset+len(cols) { + // The assign expression is for another table, not this. + continue + } val, err := asgn.Expr.Eval(ctx, m) if err != nil { return err } - colIndex := tcols[i].Offset + colIndex := i - offset touched[colIndex] = true data[colIndex] = val + assignExists = true + } + + // no assign list for this table, no need to update. + if !assignExists { + return nil } // Check whether new value is valid. @@ -198,7 +194,7 @@ func updateRecord(ctx context.Context, h int64, data []interface{}, t table.Tabl return errors.Trace(err) } // Record affected rows. - if len(insertData) == 0 { + if !onDuplicateUpdate { variable.GetSessionVars(ctx).AddAffectedRows(1) } else { variable.GetSessionVars(ctx).AddAffectedRows(2) @@ -249,17 +245,17 @@ func (s *UpdateStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) { } defer p.Close() updatedRowKeys := make(map[string]bool) - // For single-table syntax, TableRef may contain multiple tables - isMultipleTable := s.MultipleTable || s.TableRefs.MultipleTable() // Get table alias map. fs := p.GetFields() - tblAliasMap := make(map[string]string) - for _, f := range fs { - if f.TableName != f.OrgTableName { - tblAliasMap[f.TableName] = f.OrgTableName - } + + columns, err0 := getUpdateColumns(s.List, fs) + if err0 != nil { + return nil, errors.Trace(err0) } + + m := map[interface{}]interface{}{} + var records []*plan.Row for { row, err1 := p.Next(ctx) if err1 != nil { @@ -268,21 +264,30 @@ func (s *UpdateStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) { if row == nil { break } - rowData := row.Data if len(row.RowKeys) == 0 { // Nothing to update - return nil, nil + continue } + records = append(records, row) + } + + for _, row := range records { + rowData := row.Data + // Set EvalIdentFunc - m := make(map[interface{}]interface{}) m[expression.ExprEvalIdentFunc] = func(name string) (interface{}, error) { return plans.GetIdentValue(name, p.GetFields(), rowData, field.DefaultFieldFlag) } + // Update rows - start := 0 + offset := 0 for _, entry := range row.RowKeys { tbl := entry.Tbl k := entry.Key + lastOffset := offset + offset += len(tbl.Cols()) + data := rowData[lastOffset:offset] + _, ok := updatedRowKeys[k] if ok { // Each matching row is updated once, even if it matches the conditions multiple times. @@ -293,23 +298,12 @@ func (s *UpdateStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) { if err2 != nil { return nil, errors.Trace(err2) } - end := start + len(tbl.Cols()) - data := rowData[start:end] - start = end - // For multiple table mode, get to-update cols and to-update assginments. - tcols, tAsgns, err2 := getUpdateColumns(tbl, s.List, isMultipleTable, tblAliasMap) - if err2 != nil { - return nil, errors.Trace(err2) - } - if len(tcols) == 0 { - // Nothing to update for this table. - continue - } - // Get data in the table - err2 = updateRecord(ctx, handle, data, tbl, tcols, tAsgns, nil, m) + + err2 = updateRecord(ctx, handle, data, tbl, columns, m, lastOffset, false) if err2 != nil { return nil, errors.Trace(err2) } + updatedRowKeys[k] = true } } diff --git a/stmt/stmts/update_test.go b/stmt/stmts/update_test.go index b0e8888864..8582faf65f 100644 --- a/stmt/stmts/update_test.go +++ b/stmt/stmts/update_test.go @@ -179,11 +179,15 @@ func (s *testStmtSuite) TestMultipleTableUpdate(c *C) { func (s *testStmtSuite) TestIssue345(c *C) { testDB, err := sql.Open(tidb.DriverName, tidb.EngineGoLevelDBMemory+"tmp-issue345/"+s.dbName) c.Assert(err, IsNil) + mustExec(c, testDB, `drop table if exists t1, t2`) mustExec(c, testDB, `create table t1 (c1 int);`) mustExec(c, testDB, `create table t2 (c2 int);`) mustExec(c, testDB, `insert into t1 values (1);`) mustExec(c, testDB, `insert into t2 values (2);`) mustExec(c, testDB, `update t1, t2 set t1.c1 = 2, t2.c2 = 1;`) + mustExec(c, testDB, `update t1, t2 set c1 = 2, c2 = 1;`) + mustExec(c, testDB, `update t1 as a, t2 as b set a.c1 = 2, b.c2 = 1;`) + // Check t1 content tx := mustBegin(c, testDB) rows, err := tx.Query("SELECT * FROM t1;") @@ -212,5 +216,34 @@ func (s *testStmtSuite) TestIssue345(c *C) { c.Assert(err, IsNil) matchRows(c, rows, [][]interface{}{{2}}) rows.Close() + + _, err = testDB.Exec(`update t1 as a, t2 set t1.c1 = 10;`) + c.Assert(err, NotNil) + mustCommit(c, tx) } + +func (s *testStmtSuite) TestMultiUpdate(c *C) { + // fix https://github.com/pingcap/tidb/issues/369 + testSQL := ` + DROP TABLE IF EXISTS t1, t2; + create table t1 (c int); + create table t2 (c varchar(256)); + insert into t1 values (1), (2); + insert into t2 values ("a"), ("b"); + update t1, t2 set t1.c = 10, t2.c = "abc";` + mustExec(c, s.testDB, testSQL) + + // fix https://github.com/pingcap/tidb/issues/376 + testSQL = `DROP TABLE IF EXISTS t1, t2; + create table t1 (c1 int); + create table t2 (c2 int); + insert into t1 values (1), (2); + insert into t2 values (1), (2); + update t1, t2 set t1.c1 = 10, t2.c2 = 2 where t2.c2 = 1;` + mustExec(c, s.testDB, testSQL) + + rows, err := s.testDB.Query("select * from t1") + c.Assert(err, IsNil) + matchRows(c, rows, [][]interface{}{{10}, {10}}) +} diff --git a/store/localstore/kv.go b/store/localstore/kv.go index 4df0665fdd..4f34a979db 100644 --- a/store/localstore/kv.go +++ b/store/localstore/kv.go @@ -103,7 +103,6 @@ func (s *dbStore) GetSnapshot() (kv.MvccSnapshot, error) { if err != nil { return nil, errors.Trace(err) } - // dbSnapshot implements MvccSnapshot interface. return &dbSnapshot{ db: s.db, version: currentVer, @@ -121,13 +120,13 @@ func (s *dbStore) Begin() (kv.Transaction, error) { } txn := &dbTxn{ startTs: time.Now(), - tID: beginVer.Ver, + tid: beginVer.Ver, valid: true, store: s, version: kv.MinVersion, snapshotVals: make(map[string][]byte), } - log.Debugf("Begin txn:%d", txn.tID) + log.Debugf("Begin txn:%d", txn.tid) txn.UnionStore, err = kv.NewUnionStore(&dbSnapshot{ db: s.db, version: beginVer, @@ -164,7 +163,7 @@ func (s *dbStore) newBatch() engine.Batch { } // Both lock and unlock are used for simulating scenario of percolator papers. -func (s *dbStore) tryConditionLockKey(tID uint64, key string, snapshotVal []byte) error { +func (s *dbStore) tryConditionLockKey(tid uint64, key string, snapshotVal []byte) error { s.mu.Lock() defer s.mu.Unlock() @@ -187,12 +186,12 @@ func (s *dbStore) tryConditionLockKey(tID uint64, key string, snapshotVal []byte } // If there's newer version of this key, returns error. - if ver > tID { - log.Warnf("txn:%d, tryLockKey condition not match for key %s, currValue:%q, snapshotVal:%q", tID, key, currValue, snapshotVal) + if ver > tid { + log.Warnf("txn:%d, tryLockKey condition not match for key %s, currValue:%q, snapshotVal:%q", tid, key, currValue, snapshotVal) return errors.Trace(kv.ErrConditionNotMatch) } - s.keysLocked[key] = tID + s.keysLocked[key] = tid return nil } diff --git a/store/localstore/local_version_provider.go b/store/localstore/local_version_provider.go index 1502f46016..2b35e7360a 100644 --- a/store/localstore/local_version_provider.go +++ b/store/localstore/local_version_provider.go @@ -28,6 +28,7 @@ const ( func (l *LocalVersionProvider) CurrentVersion() (kv.Version, error) { l.mu.Lock() defer l.mu.Unlock() + var ts uint64 ts = uint64((time.Now().UnixNano() / int64(time.Millisecond)) << timePrecisionOffset) if l.lastTimeStampTs == uint64(ts) { diff --git a/store/localstore/snapshot.go b/store/localstore/snapshot.go index 6d52a49978..ff53e90511 100644 --- a/store/localstore/snapshot.go +++ b/store/localstore/snapshot.go @@ -29,6 +29,7 @@ var ( _ kv.Iterator = (*dbIter)(nil) ) +// dbSnapshot implements MvccSnapshot interface. type dbSnapshot struct { db engine.DB rawIt engine.Iterator diff --git a/store/localstore/txn.go b/store/localstore/txn.go index 04d00ecf81..27a254dff0 100644 --- a/store/localstore/txn.go +++ b/store/localstore/txn.go @@ -39,7 +39,7 @@ type dbTxn struct { kv.UnionStore store *dbStore // for commit startTs time.Time - tID uint64 + tid uint64 valid bool version kv.Version // commit version snapshotVals map[string][]byte // origin version in snapshot @@ -66,7 +66,7 @@ func (txn *dbTxn) markOrigin(k []byte) error { // Implement transaction interface func (txn *dbTxn) Inc(k kv.Key, step int64) (int64, error) { - log.Debugf("Inc %q, step %d txn:%d", k, step, txn.tID) + log.Debugf("Inc %q, step %d txn:%d", k, step, txn.tid) k = kv.EncodeKey(k) if err := txn.markOrigin(k); err != nil { @@ -113,11 +113,11 @@ func (txn *dbTxn) GetInt64(k kv.Key) (int64, error) { } func (txn *dbTxn) String() string { - return fmt.Sprintf("%d", txn.tID) + return fmt.Sprintf("%d", txn.tid) } func (txn *dbTxn) Get(k kv.Key) ([]byte, error) { - log.Debugf("get key:%q, txn:%d", k, txn.tID) + log.Debugf("get key:%q, txn:%d", k, txn.tid) k = kv.EncodeKey(k) val, err := txn.UnionStore.Get(k) if kv.IsErrNotFound(err) { @@ -140,7 +140,7 @@ func (txn *dbTxn) Set(k kv.Key, data []byte) error { return errors.Trace(ErrCannotSetNilValue) } - log.Debugf("set key:%q, txn:%d", k, txn.tID) + log.Debugf("set key:%q, txn:%d", k, txn.tid) k = kv.EncodeKey(k) err := txn.UnionStore.Set(k, data) if err != nil { @@ -151,7 +151,7 @@ func (txn *dbTxn) Set(k kv.Key, data []byte) error { } func (txn *dbTxn) Seek(k kv.Key, fnKeyCmp func(kv.Key) bool) (kv.Iterator, error) { - log.Debugf("seek key:%q, txn:%d", k, txn.tID) + log.Debugf("seek key:%q, txn:%d", k, txn.tid) k = kv.EncodeKey(k) iter, err := txn.UnionStore.Seek(k, txn) @@ -172,7 +172,7 @@ func (txn *dbTxn) Seek(k kv.Key, fnKeyCmp func(kv.Key) bool) (kv.Iterator, error } func (txn *dbTxn) Delete(k kv.Key) error { - log.Debugf("delete key:%q, txn:%d", k, txn.tID) + log.Debugf("delete key:%q, txn:%d", k, txn.tid) k = kv.EncodeKey(k) return txn.UnionStore.Delete(k) } @@ -198,7 +198,7 @@ func (txn *dbTxn) doCommit() error { }() // Check locked keys for k, v := range txn.snapshotVals { - err := txn.store.tryConditionLockKey(txn.tID, k, v) + err := txn.store.tryConditionLockKey(txn.tid, k, v) if err != nil { return errors.Trace(err) } @@ -236,7 +236,7 @@ func (txn *dbTxn) Commit() error { if !txn.valid { return errors.Trace(ErrInvalidTxn) } - log.Infof("commit txn %d", txn.tID) + log.Infof("commit txn %d", txn.tid) defer func() { txn.close() }() @@ -263,7 +263,7 @@ func (txn *dbTxn) Rollback() error { if !txn.valid { return errors.Trace(ErrInvalidTxn) } - log.Warnf("Rollback txn %d", txn.tID) + log.Warnf("Rollback txn %d", txn.tid) return txn.close() } diff --git a/tidb-server/server/conn.go b/tidb-server/server/conn.go index c1056d917c..898f91bd84 100644 --- a/tidb-server/server/conn.go +++ b/tidb-server/server/conn.go @@ -201,11 +201,11 @@ func (cc *clientConn) readHandshakeResponse() error { addr := cc.conn.RemoteAddr().String() host, _, err1 := net.SplitHostPort(addr) if err1 != nil { - return errors.Trace(mysql.NewDefaultError(mysql.ErAccessDeniedError, cc.user, addr, "Yes")) + return errors.Trace(mysql.NewErr(mysql.ErrAccessDenied, cc.user, addr, "Yes")) } user := fmt.Sprintf("%s@%s", cc.user, host) if !cc.ctx.Auth(user, auth, cc.salt) { - return errors.Trace(mysql.NewDefaultError(mysql.ErAccessDeniedError, cc.user, host, "Yes")) + return errors.Trace(mysql.NewErr(mysql.ErrAccessDenied, cc.user, host, "Yes")) } } return nil @@ -283,8 +283,7 @@ func (cc *clientConn) dispatch(data []byte) error { case mysql.ComStmtReset: return cc.handleStmtReset(data) default: - msg := fmt.Sprintf("command %d not supported now", cmd) - return mysql.NewError(mysql.ErUnknownError, msg) + return mysql.NewErrf(mysql.ErrUnknown, "command %d not supported now", cmd) } } @@ -322,8 +321,9 @@ func (cc *clientConn) writeOK() error { func (cc *clientConn) writeError(e error) error { var m *mysql.SQLError var ok bool - if m, ok = e.(*mysql.SQLError); !ok { - m = mysql.NewError(mysql.ErUnknownError, e.Error()) + originErr := errors.Cause(e) + if m, ok = originErr.(*mysql.SQLError); !ok { + m = mysql.NewErrf(mysql.ErrUnknown, e.Error()) } data := make([]byte, 4, 16+len(m.Message)) diff --git a/tidb-server/server/conn_stmt.go b/tidb-server/server/conn_stmt.go index 24b49c33fe..aacaf4933a 100644 --- a/tidb-server/server/conn_stmt.go +++ b/tidb-server/server/conn_stmt.go @@ -36,7 +36,6 @@ package server import ( "encoding/binary" - "fmt" "math" "strconv" @@ -113,7 +112,7 @@ func (cc *clientConn) handleStmtExecute(data []byte) (err error) { stmt := cc.ctx.GetStatement(int(stmtID)) if stmt == nil { - return mysql.NewDefaultError(mysql.ErUnknownStmtHandler, + return mysql.NewErr(mysql.ErrUnknownStmtHandler, strconv.FormatUint(uint64(stmtID), 10), "stmt_execute") } @@ -121,7 +120,7 @@ func (cc *clientConn) handleStmtExecute(data []byte) (err error) { pos++ //now we only support CURSOR_TYPE_NO_CURSOR flag if flag != 0 { - return mysql.NewError(mysql.ErUnknownError, fmt.Sprintf("unsupported flag %d", flag)) + return mysql.NewErrf(mysql.ErrUnknown, "unsupported flag %d", flag) } //skip iteration-count, always 1 @@ -324,7 +323,7 @@ func (cc *clientConn) handleStmtSendLongData(data []byte) (err error) { stmt := cc.ctx.GetStatement(stmtID) if stmt == nil { - return mysql.NewDefaultError(mysql.ErUnknownStmtHandler, + return mysql.NewErr(mysql.ErrUnknownStmtHandler, strconv.Itoa(stmtID), "stmt_send_longdata") } @@ -340,7 +339,7 @@ func (cc *clientConn) handleStmtReset(data []byte) (err error) { stmtID := int(binary.LittleEndian.Uint32(data[0:4])) stmt := cc.ctx.GetStatement(stmtID) if stmt == nil { - return mysql.NewDefaultError(mysql.ErUnknownStmtHandler, + return mysql.NewErr(mysql.ErrUnknownStmtHandler, strconv.Itoa(stmtID), "stmt_reset") } stmt.Reset() diff --git a/tidb-server/server/driver_tidb.go b/tidb-server/server/driver_tidb.go index 1ea32908fa..79c23ae93c 100644 --- a/tidb-server/server/driver_tidb.go +++ b/tidb-server/server/driver_tidb.go @@ -21,6 +21,7 @@ import ( mysql "github.com/pingcap/tidb/mysqldef" "github.com/pingcap/tidb/rset" "github.com/pingcap/tidb/util/errors2" + "github.com/pingcap/tidb/util/types" ) // TiDBDriver implements IDriver. @@ -78,7 +79,7 @@ func (ts *TiDBStatement) Execute(args ...interface{}) (rs ResultSet, err error) // AppendParam implements IStatement AppendParam method. func (ts *TiDBStatement) AppendParam(paramID int, data []byte) error { if paramID >= len(ts.boundParams) { - return mysql.NewDefaultError(mysql.ErWrongArguments, "stmt_send_longdata") + return mysql.NewErr(mysql.ErrWrongArguments, "stmt_send_longdata") } ts.boundParams[paramID] = append(ts.boundParams[paramID], data...) return nil @@ -272,8 +273,16 @@ func convertColumnInfo(fld *field.ResultField) (ci *ColumnInfo) { ci.Schema = fld.DBName ci.Flag = uint16(fld.Flag) ci.Charset = uint16(mysql.CharsetIDs[fld.Charset]) - ci.ColumnLength = uint32(fld.Flen) - ci.Decimal = uint8(fld.Decimal) + if fld.Flen == types.UnspecifiedLength { + ci.ColumnLength = 0 + } else { + ci.ColumnLength = uint32(fld.Flen) + } + if fld.Decimal == types.UnspecifiedLength { + ci.Decimal = 0 + } else { + ci.Decimal = uint8(fld.Decimal) + } ci.Type = uint8(fld.Tp) return } diff --git a/tidb.go b/tidb.go index f78a8bbcd0..dbe455b4b2 100644 --- a/tidb.go +++ b/tidb.go @@ -26,11 +26,13 @@ import ( "github.com/juju/errors" "github.com/ngaut/log" + "github.com/pingcap/tidb/ast" "github.com/pingcap/tidb/context" "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/field" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/optimizer" "github.com/pingcap/tidb/parser" "github.com/pingcap/tidb/rset" "github.com/pingcap/tidb/sessionctx/variable" @@ -92,8 +94,20 @@ func Compile(src string) ([]stmt.Statement, error) { log.Warnf("compiling %s, error: %v", src, l.Errors()[0]) return nil, errors.Trace(l.Errors()[0]) } - - return l.Stmts(), nil + rawStmt := l.Stmts() + stmts := make([]stmt.Statement, len(rawStmt)) + for i, v := range rawStmt { + if node, ok := v.(ast.Node); ok { + stm, err := optimizer.Compile(node) + if err != nil { + return nil, errors.Trace(err) + } + stmts[i] = stm + } else { + stmts[i] = v.(stmt.Statement) + } + } + return stmts, nil } // CompilePrepare compiles prepared statement, allows placeholder as expr. @@ -112,7 +126,7 @@ func CompilePrepare(src string) (stmt.Statement, []*expression.ParamMarker, erro return nil, nil, nil } sm := sms[0] - return sm, l.ParamList, nil + return sm.(stmt.Statement), l.ParamList, nil } func prepareStmt(ctx context.Context, sqlText string) (stmtID uint32, paramCount int, fields []*field.ResultField, err error) { diff --git a/tidb_test.go b/tidb_test.go index b14b5fdbad..aa082ab6a7 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -735,6 +735,21 @@ func (s *testSessionSuite) TestIndex(c *C) { c.Assert(err, IsNil) c.Assert(rows, HasLen, 1) match(c, rows[0], 1) + + mustExecSQL(c, se, "drop table if exists t1, t2") + mustExecSQL(c, se, ` + create table t1 (c1 int, primary key(c1)); + create table t2 (c2 int, primary key(c2)); + insert into t1 values (1), (2); + insert into t2 values (2);`) + + r = mustExecSQL(c, se, "select * from t1 left join t2 on t1.c1 = t2.c2 order by t1.c1") + rows, err = r.Rows(-1, 0) + matches(c, rows, [][]interface{}{{1, nil}, {2, 2}}) + + r = mustExecSQL(c, se, "select * from t1 left join t2 on t1.c1 = t2.c2 where t2.c2 < 10") + rows, err = r.Rows(-1, 0) + matches(c, rows, [][]interface{}{{2, 2}}) } func (s *testSessionSuite) TestMySQLTypes(c *C) { @@ -792,6 +807,11 @@ func (s *testSessionSuite) TestSelect(c *C) { c.Assert(err, IsNil) match(c, row, 1, 2) + r = mustExecSQL(c, se, "select 1, 2 from dual where not exists (select * from t where c1=2)") + row, err = r.FirstRow() + c.Assert(err, IsNil) + match(c, row, 1, 2) + r = mustExecSQL(c, se, "select 1, 2") row, err = r.FirstRow() c.Assert(err, IsNil) @@ -848,6 +868,20 @@ func (s *testSessionSuite) TestSelect(c *C) { row, err = r.FirstRow() c.Assert(err, IsNil) c.Assert(row, IsNil) + + mustExecSQL(c, se, "drop table if exists t1, t2, t3") + mustExecSQL(c, se, ` + create table t1 (c1 int); + create table t2 (c2 int); + create table t3 (c3 int); + insert into t1 values (1), (2); + insert into t2 values (2); + insert into t3 values (3);`) + r = mustExecSQL(c, se, "select * from t1 left join t2 on t1.c1 = t2.c2 left join t3 on t1.c1 = t3.c3 order by t1.c1") + rows, err = r.Rows(-1, 0) + c.Assert(err, IsNil) + matches(c, rows, [][]interface{}{{1, nil, nil}, {2, 2, nil}}) + } func (s *testSessionSuite) TestSubQuery(c *C) { @@ -870,6 +904,8 @@ func (s *testSessionSuite) TestSubQuery(c *C) { c.Assert(rows, HasLen, 2) match(c, rows[0], 0) match(c, rows[1], 2) + + mustExecMatch(c, se, "select a.c1, a.c2 from (select c1 as c1, c1 as c2 from t1) as a", [][]interface{}{{1, 1}, {2, 2}}) } func (s *testSessionSuite) TestShow(c *C) { @@ -888,7 +924,7 @@ func (s *testSessionSuite) TestShow(c *C) { rows, err := r.Rows(-1, 0) c.Assert(err, IsNil) c.Assert(rows, HasLen, 1) - match(c, rows[0], "c", "int", "YES", "", nil, "") + match(c, rows[0], "c", "int(11)", "YES", "", nil, "") r = mustExecSQL(c, se, "show collation where Charset = 'utf8' and Collation = 'utf8_bin'") row, err = r.FirstRow() @@ -1067,6 +1103,86 @@ func (s *testSessionSuite) TestWhereLike(c *C) { c.Assert(rows, HasLen, 6) } +func (s *testSessionSuite) TestDefaultFlenBug(c *C) { + // If set unspecified column flen to 0, it will cause bug in union. + // This test is used to prevent the bug reappear. + store := newStore(c, s.dbName) + se := newSession(c, store, s.dbName) + + mustExecSQL(c, se, "create table t1 (c double);") + mustExecSQL(c, se, "create table t2 (c double);") + mustExecSQL(c, se, "insert into t1 value (73);") + mustExecSQL(c, se, "insert into t2 value (930);") + // The data in the second src will be casted as the type of the first src. + // If use flen=0, it will be truncated. + r := mustExecSQL(c, se, "select c from t1 union select c from t2;") + rows, err := r.Rows(-1, 0) + c.Assert(err, IsNil) + c.Assert(rows, HasLen, 2) + c.Assert(rows[1][0], Equals, float64(930)) +} + +func (s *testSessionSuite) TestExecRestrictedSQL(c *C) { + store := newStore(c, s.dbName) + se := newSession(c, store, s.dbName).(*session) + r, err := se.ExecRestrictedSQL(se, "select 1;") + c.Assert(r, NotNil) + c.Assert(err, IsNil) + _, err = se.ExecRestrictedSQL(se, "select 1; select 2;") + c.Assert(err, NotNil) + _, err = se.ExecRestrictedSQL(se, "") + c.Assert(err, NotNil) +} + +func (s *testSessionSuite) TestGroupBy(c *C) { + store := newStore(c, s.dbName) + se := newSession(c, store, s.dbName) + mustExecSQL(c, se, "drop table if exists t") + mustExecSQL(c, se, "create table t (c1 int, c2 int)") + mustExecSQL(c, se, "insert into t values (1,1), (2,2), (1,2), (1,3)") + + mustExecMatch(c, se, "select c1 as c2, c2 from t group by c2 + 1", [][]interface{}{{1, 1}, {2, 2}, {1, 3}}) + mustExecMatch(c, se, "select c1 as c2, count(c1) from t group by c2", [][]interface{}{{1, 1}, {2, 2}, {1, 1}}) + mustExecMatch(c, se, "select t.c1, c1 from t group by c1", [][]interface{}{{1, 1}, {2, 2}}) + mustExecMatch(c, se, "select t.c1 as a, c1 as a from t group by a", [][]interface{}{{1, 1}, {2, 2}}) + + mustExecFailed(c, se, "select c1 as a, c2 as a from t group by a") + mustExecFailed(c, se, "select c1 as c2, c2 from t group by c2") + mustExecFailed(c, se, "select sum(c1) as a from t group by a") + mustExecFailed(c, se, "select sum(c1) as a from t group by a + 1") +} + +func (s *testSessionSuite) TestOrderBy(c *C) { + store := newStore(c, s.dbName) + se := newSession(c, store, s.dbName) + mustExecSQL(c, se, "drop table if exists t") + mustExecSQL(c, se, "create table t (c1 int, c2 int)") + mustExecSQL(c, se, "insert into t values (1,2), (2, 1)") + + // Fix issue https://github.com/pingcap/tidb/issues/337 + mustExecMatch(c, se, "select c1 as a, c1 as b from t order by c1", [][]interface{}{{1, 1}, {2, 2}}) + + mustExecMatch(c, se, "select c1 as a, t.c1 as a from t order by a desc", [][]interface{}{{2, 2}, {1, 1}}) + mustExecMatch(c, se, "select c1 as c2 from t order by c2", [][]interface{}{{1}, {2}}) + mustExecMatch(c, se, "select sum(c1) from t order by sum(c1)", [][]interface{}{{3}}) + + // TODO: now this test result is not same as MySQL, we will update it later. + mustExecMatch(c, se, "select c1 as c2 from t order by c2 + 1", [][]interface{}{{1}, {2}}) + + mustExecFailed(c, se, "select c1 as a, c2 as a from t order by a") +} + +func (s *testSessionSuite) TestHaving(c *C) { + store := newStore(c, s.dbName) + se := newSession(c, store, s.dbName) + mustExecSQL(c, se, "drop table if exists t") + mustExecSQL(c, se, "create table t (c1 int, c2 int)") + mustExecSQL(c, se, "insert into t values (1,2), (2, 1)") + + mustExecMatch(c, se, "select sum(c1) from t group by c1 having sum(c1)", [][]interface{}{{1}, {2}}) + mustExecMatch(c, se, "select sum(c1) - 1 from t group by c1 having sum(c1) - 1", [][]interface{}{{1}}) +} + func newSession(c *C, store kv.Storage, dbName string) Session { se, err := CreateSession(store) c.Assert(err, IsNil) @@ -1131,3 +1247,22 @@ func match(c *C, row []interface{}, expected ...interface{}) { c.Assert(got, Equals, need) } } + +func matches(c *C, rows [][]interface{}, expected [][]interface{}) { + c.Assert(len(rows), Equals, len(expected)) + for i := 0; i < len(rows); i++ { + match(c, rows[i], expected[i]...) + } +} + +func mustExecMatch(c *C, se Session, sql string, expected [][]interface{}) { + r := mustExecSQL(c, se, sql) + rows, err := r.Rows(-1, 0) + c.Assert(err, IsNil) + matches(c, rows, expected) +} + +func mustExecFailed(c *C, se Session, sql string, args ...interface{}) { + _, err := exec(c, se, sql, args...) + c.Assert(err, NotNil) +} diff --git a/util/sqlexec/restricted_sql_executor.go b/util/sqlexec/restricted_sql_executor.go new file mode 100644 index 0000000000..c0aa7a5fe0 --- /dev/null +++ b/util/sqlexec/restricted_sql_executor.go @@ -0,0 +1,44 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package sqlexec + +import ( + "github.com/pingcap/tidb/context" + "github.com/pingcap/tidb/rset" +) + +// RestrictedSQLExecutorKeyType is a dummy type to avoid naming collision in session. +type RestrictedSQLExecutorKeyType struct{} + +// String implements Stringer interface. +func (k *RestrictedSQLExecutorKeyType) String() string { + return "restricted_sql_executor" +} + +// RestrictedSQLExecutor is an interface provides executing restricted sql statement. +// Why we need this interface? +// When we execute some management statements, we need to operate system tables. +// For example when executing create user statement, we need to check if the user already +// exists in the mysql.User table and insert a new row if not exists. In this case, we need +// a convenience way to manipulate system tables. The most simple way is executing sql statement. +// In order to execute sql statement in stmts package, we add this interface to solve dependence problem. +// And in the same time, we do not want this interface becomes a general way to run sql statement. +// We hope this could be used with some restrictions such as only allowing system tables as target, +// do not allowing recursion call. +// For more infomation please refer to the comments in session.ExecRestrictedSQL(). +type RestrictedSQLExecutor interface { + // ExecRestrictedSQL run sql statement in ctx with some restriction. + // This is implemented in session.go. + ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recordset, error) +} diff --git a/util/types/convert_test.go b/util/types/convert_test.go index 892eabcf1d..300d54aea6 100644 --- a/util/types/convert_test.go +++ b/util/types/convert_test.go @@ -447,11 +447,11 @@ func (s *testTypeConvertSuite) TestStrToNum(c *C) { } func (s *testTypeConvertSuite) TestFieldTypeToStr(c *C) { - v := FieldTypeToStr(mysql.TypeDecimal, "not binary") + v := TypeToStr(mysql.TypeDecimal, "not binary") c.Assert(v, Equals, type2Str[mysql.TypeDecimal]) - v = FieldTypeToStr(mysql.TypeBlob, charset.CharsetBin) + v = TypeToStr(mysql.TypeBlob, charset.CharsetBin) c.Assert(v, Equals, "blob") - v = FieldTypeToStr(mysql.TypeString, charset.CharsetBin) + v = TypeToStr(mysql.TypeString, charset.CharsetBin) c.Assert(v, Equals, "binary") } diff --git a/util/types/etc.go b/util/types/etc.go index 7387e25515..1df8a83eff 100644 --- a/util/types/etc.go +++ b/util/types/etc.go @@ -23,7 +23,6 @@ import ( "strings" "github.com/juju/errors" - "github.com/ngaut/log" mysql "github.com/pingcap/tidb/mysqldef" "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/util/charset" @@ -85,84 +84,13 @@ func TypeStr(tp byte) (r string) { return type2Str[tp] } -// TypeToStr converts tp to a string with an extra binary. -func TypeToStr(tp byte, binary bool) string { - switch tp { - case mysql.TypeBlob: - if binary { - return "text" - } - return "blob" - case mysql.TypeLongBlob: - if binary { - return "longtext" - } - return "longblob" - case mysql.TypeTinyBlob: - if binary { - return "tinytext" - } - return "tinyblob" - case mysql.TypeMediumBlob: - if binary { - return "mediumtext" - } - return "mediumblob" - case mysql.TypeVarchar: - if binary { - return "varbinary" - } - return "varchar" - case mysql.TypeString: - if binary { - return "binary" - } - return "char" - case mysql.TypeTiny: - return "tinyint" - case mysql.TypeShort: - return "smallint" - case mysql.TypeInt24: - return "mediumint" - case mysql.TypeLong: - return "int" - case mysql.TypeLonglong: - return "bigint" - case mysql.TypeFloat: - return "float" - case mysql.TypeDouble: - return "double" - case mysql.TypeDecimal, mysql.TypeNewDecimal: - return "decimal" - case mysql.TypeYear: - return "year" - case mysql.TypeDuration: - return "time" - case mysql.TypeDatetime: - return "datetime" - case mysql.TypeDate: - return "date" - case mysql.TypeTimestamp: - return "timestamp" - case mysql.TypeBit: - return "bit" - case mysql.TypeEnum: - return "enum" - case mysql.TypeSet: - return "set" - default: - log.Errorf("unkown type %d, binary %v", tp, binary) - } - return "" -} - -// FieldTypeToStr converts a field to a string. +// TypeToStr converts a field to a string. // It is used for converting Text to Blob, // or converting Char to Binary. // Args: // tp: type enum // cs: charset -func FieldTypeToStr(tp byte, cs string) (r string) { +func TypeToStr(tp byte, cs string) (r string) { ts := type2Str[tp] if cs != charset.CharsetBin { return ts diff --git a/util/types/etc_test.go b/util/types/etc_test.go index 6212535605..20aa8d6626 100644 --- a/util/types/etc_test.go +++ b/util/types/etc_test.go @@ -59,8 +59,8 @@ func testTypeStr(c *C, tp byte, expect string) { c.Assert(v, Equals, expect) } -func testTypeToStr(c *C, tp byte, binary bool, expect string) { - v := TypeToStr(tp, binary) +func testTypeToStr(c *C, tp byte, charset string, expect string) { + v := TypeToStr(tp, charset) c.Assert(v, Equals, expect) } @@ -68,36 +68,36 @@ func (s *testTypeEtcSuite) TestTypeToStr(c *C) { testTypeStr(c, mysql.TypeYear, "year") testTypeStr(c, 0xdd, "") - testTypeToStr(c, mysql.TypeBlob, true, "text") - testTypeToStr(c, mysql.TypeLongBlob, true, "longtext") - testTypeToStr(c, mysql.TypeTinyBlob, true, "tinytext") - testTypeToStr(c, mysql.TypeMediumBlob, true, "mediumtext") - testTypeToStr(c, mysql.TypeVarchar, true, "varbinary") - testTypeToStr(c, mysql.TypeString, true, "binary") - testTypeToStr(c, mysql.TypeTiny, true, "tinyint") - testTypeToStr(c, mysql.TypeBlob, false, "blob") - testTypeToStr(c, mysql.TypeLongBlob, false, "longblob") - testTypeToStr(c, mysql.TypeTinyBlob, false, "tinyblob") - testTypeToStr(c, mysql.TypeMediumBlob, false, "mediumblob") - testTypeToStr(c, mysql.TypeVarchar, false, "varchar") - testTypeToStr(c, mysql.TypeString, false, "char") - testTypeToStr(c, mysql.TypeShort, true, "smallint") - testTypeToStr(c, mysql.TypeInt24, true, "mediumint") - testTypeToStr(c, mysql.TypeLong, true, "int") - testTypeToStr(c, mysql.TypeLonglong, true, "bigint") - testTypeToStr(c, mysql.TypeFloat, true, "float") - testTypeToStr(c, mysql.TypeDouble, true, "double") - testTypeToStr(c, mysql.TypeYear, true, "year") - testTypeToStr(c, mysql.TypeDuration, true, "time") - testTypeToStr(c, mysql.TypeDatetime, true, "datetime") - testTypeToStr(c, mysql.TypeDate, true, "date") - testTypeToStr(c, mysql.TypeTimestamp, true, "timestamp") - testTypeToStr(c, mysql.TypeNewDecimal, true, "decimal") - testTypeToStr(c, mysql.TypeDecimal, true, "decimal") - testTypeToStr(c, 0xdd, true, "") - testTypeToStr(c, mysql.TypeBit, true, "bit") - testTypeToStr(c, mysql.TypeEnum, true, "enum") - testTypeToStr(c, mysql.TypeSet, true, "set") + testTypeToStr(c, mysql.TypeBlob, "utf8", "text") + testTypeToStr(c, mysql.TypeLongBlob, "utf8", "longtext") + testTypeToStr(c, mysql.TypeTinyBlob, "utf8", "tinytext") + testTypeToStr(c, mysql.TypeMediumBlob, "utf8", "mediumtext") + testTypeToStr(c, mysql.TypeVarchar, "binary", "varbinary") + testTypeToStr(c, mysql.TypeString, "binary", "binary") + testTypeToStr(c, mysql.TypeTiny, "binary", "tinyint") + testTypeToStr(c, mysql.TypeBlob, "binary", "blob") + testTypeToStr(c, mysql.TypeLongBlob, "binary", "longblob") + testTypeToStr(c, mysql.TypeTinyBlob, "binary", "tinyblob") + testTypeToStr(c, mysql.TypeMediumBlob, "binary", "mediumblob") + testTypeToStr(c, mysql.TypeVarchar, "utf8", "varchar") + testTypeToStr(c, mysql.TypeString, "utf8", "char") + testTypeToStr(c, mysql.TypeShort, "binary", "smallint") + testTypeToStr(c, mysql.TypeInt24, "binary", "mediumint") + testTypeToStr(c, mysql.TypeLong, "binary", "int") + testTypeToStr(c, mysql.TypeLonglong, "binary", "bigint") + testTypeToStr(c, mysql.TypeFloat, "binary", "float") + testTypeToStr(c, mysql.TypeDouble, "binary", "double") + testTypeToStr(c, mysql.TypeYear, "binary", "year") + testTypeToStr(c, mysql.TypeDuration, "binary", "time") + testTypeToStr(c, mysql.TypeDatetime, "binary", "datetime") + testTypeToStr(c, mysql.TypeDate, "binary", "date") + testTypeToStr(c, mysql.TypeTimestamp, "binary", "timestamp") + testTypeToStr(c, mysql.TypeNewDecimal, "binary", "decimal") + testTypeToStr(c, mysql.TypeDecimal, "binary", "decimal") + testTypeToStr(c, 0xdd, "binary", "") + testTypeToStr(c, mysql.TypeBit, "binary", "bit") + testTypeToStr(c, mysql.TypeEnum, "binary", "enum") + testTypeToStr(c, mysql.TypeSet, "binary", "set") } func (s *testTypeEtcSuite) TestEOFAsNil(c *C) { diff --git a/util/types/field_type.go b/util/types/field_type.go index 2ac6042ef6..ec96912bcf 100644 --- a/util/types/field_type.go +++ b/util/types/field_type.go @@ -52,45 +52,62 @@ func NewFieldType(tp byte) *FieldType { } } -// String joins the information of FieldType and -// returns a string. -func (ft *FieldType) String() string { - ts := FieldTypeToStr(ft.Tp, ft.Charset) - ans := []string{ts} +// CompactStr only considers Tp/CharsetBin/Flen/Deimal. +// This is used for showing column type in infoschema. +func (ft *FieldType) CompactStr() string { + ts := TypeToStr(ft.Tp, ft.Charset) + suffix := "" switch ft.Tp { case mysql.TypeEnum, mysql.TypeSet: // Format is ENUM ('e1', 'e2') or SET ('e1', 'e2') - ans = append(ans, fmt.Sprintf("('%s')", strings.Join(ft.Elems, "','"))) + es := make([]string, 0, len(ft.Elems)) + for _, e := range ft.Elems { + e = strings.Replace(e, "'", "''", -1) + es = append(es, e) + } + suffix = fmt.Sprintf("('%s')", strings.Join(es, "','")) + case mysql.TypeTimestamp, mysql.TypeDatetime, mysql.TypeDate: + if ft.Decimal != UnspecifiedLength && ft.Decimal != 0 { + suffix = fmt.Sprintf("(%d)", ft.Decimal) + } default: if ft.Flen != UnspecifiedLength { if ft.Decimal == UnspecifiedLength { - ans = append(ans, fmt.Sprintf("(%d)", ft.Flen)) + if ft.Tp != mysql.TypeFloat && ft.Tp != mysql.TypeDouble { + suffix = fmt.Sprintf("(%d)", ft.Flen) + } } else { - ans = append(ans, fmt.Sprintf("(%d, %d)", ft.Flen, ft.Decimal)) + suffix = fmt.Sprintf("(%d,%d)", ft.Flen, ft.Decimal) } } else if ft.Decimal != UnspecifiedLength { - ans = append(ans, fmt.Sprintf("(%d)", ft.Decimal)) + suffix = fmt.Sprintf("(%d)", ft.Decimal) } } + return ts + suffix +} +// String joins the information of FieldType and +// returns a string. +func (ft *FieldType) String() string { + strs := []string{ft.CompactStr()} if mysql.HasUnsignedFlag(ft.Flag) { - ans = append(ans, "UNSIGNED") + strs = append(strs, "UNSIGNED") } if mysql.HasZerofillFlag(ft.Flag) { - ans = append(ans, "ZEROFILL") + strs = append(strs, "ZEROFILL") } if mysql.HasBinaryFlag(ft.Flag) { - ans = append(ans, "BINARY") + strs = append(strs, "BINARY") } if IsTypeChar(ft.Tp) || IsTypeBlob(ft.Tp) { if ft.Charset != "" && ft.Charset != charset.CharsetBin { - ans = append(ans, fmt.Sprintf("CHARACTER SET %s", ft.Charset)) + strs = append(strs, fmt.Sprintf("CHARACTER SET %s", ft.Charset)) } if ft.Collate != "" && ft.Collate != charset.CharsetBin { - ans = append(ans, fmt.Sprintf("COLLATE %s", ft.Collate)) + strs = append(strs, fmt.Sprintf("COLLATE %s", ft.Collate)) } } - return strings.Join(ans, " ") + return strings.Join(strs, " ") } diff --git a/util/types/field_type_test.go b/util/types/field_type_test.go index 3e73cc53ed..558c799d8a 100644 --- a/util/types/field_type_test.go +++ b/util/types/field_type_test.go @@ -28,33 +28,80 @@ func (s *testFieldTypeSuite) TestFieldType(c *C) { c.Assert(ft.Flen, Equals, UnspecifiedLength) c.Assert(ft.Decimal, Equals, UnspecifiedLength) ft.Decimal = 5 - c.Assert(ft.String(), Equals, "time (5)") + c.Assert(ft.String(), Equals, "time(5)") ft.Tp = mysql.TypeLong ft.Flag |= mysql.UnsignedFlag | mysql.ZerofillFlag - c.Assert(ft.String(), Equals, "int (5) UNSIGNED ZEROFILL") + c.Assert(ft.String(), Equals, "int(5) UNSIGNED ZEROFILL") ft = NewFieldType(mysql.TypeFloat) ft.Flen = 10 ft.Decimal = 3 - c.Assert(ft.String(), Equals, "float (10, 3)") + c.Assert(ft.String(), Equals, "float(10,3)") + ft = NewFieldType(mysql.TypeFloat) + ft.Flen = 10 + ft.Decimal = -1 + c.Assert(ft.String(), Equals, "float") + + ft = NewFieldType(mysql.TypeDouble) + ft.Flen = 10 + ft.Decimal = 3 + c.Assert(ft.String(), Equals, "double(10,3)") + ft = NewFieldType(mysql.TypeDouble) + ft.Flen = 10 + ft.Decimal = -1 + c.Assert(ft.String(), Equals, "double") ft = NewFieldType(mysql.TypeBlob) ft.Flen = 10 ft.Charset = "UTF8" ft.Collate = "UTF8_UNICODE_GI" - c.Assert(ft.String(), Equals, "text (10) CHARACTER SET UTF8 COLLATE UTF8_UNICODE_GI") + c.Assert(ft.String(), Equals, "text(10) CHARACTER SET UTF8 COLLATE UTF8_UNICODE_GI") ft = NewFieldType(mysql.TypeVarchar) ft.Flen = 10 ft.Flag |= mysql.BinaryFlag - c.Assert(ft.String(), Equals, "varchar (10) BINARY") + c.Assert(ft.String(), Equals, "varchar(10) BINARY") ft = NewFieldType(mysql.TypeEnum) ft.Elems = []string{"a", "b"} - c.Assert(ft.String(), Equals, "enum ('a','b')") + c.Assert(ft.String(), Equals, "enum('a','b')") + + ft = NewFieldType(mysql.TypeEnum) + ft.Elems = []string{"'a'", "'b'"} + c.Assert(ft.String(), Equals, "enum('''a''','''b''')") ft = NewFieldType(mysql.TypeSet) ft.Elems = []string{"a", "b"} - c.Assert(ft.String(), Equals, "set ('a','b')") + c.Assert(ft.String(), Equals, "set('a','b')") + + ft = NewFieldType(mysql.TypeSet) + ft.Elems = []string{"'a'", "'b'"} + c.Assert(ft.String(), Equals, "set('''a''','''b''')") + + ft = NewFieldType(mysql.TypeTimestamp) + ft.Flen = 8 + ft.Decimal = 2 + c.Assert(ft.String(), Equals, "timestamp(2)") + ft = NewFieldType(mysql.TypeTimestamp) + ft.Flen = 8 + ft.Decimal = 0 + c.Assert(ft.String(), Equals, "timestamp") + + ft = NewFieldType(mysql.TypeDatetime) + ft.Flen = 8 + ft.Decimal = 2 + c.Assert(ft.String(), Equals, "datetime(2)") + ft = NewFieldType(mysql.TypeDatetime) + ft.Flen = 8 + ft.Decimal = 0 + c.Assert(ft.String(), Equals, "datetime") + ft = NewFieldType(mysql.TypeDate) + ft.Flen = 8 + ft.Decimal = 2 + c.Assert(ft.String(), Equals, "date(2)") + ft = NewFieldType(mysql.TypeDate) + ft.Flen = 8 + ft.Decimal = 0 + c.Assert(ft.String(), Equals, "date") }