From d121489be08ef4691727a086b52e77adfcc87333 Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Mon, 12 Oct 2015 14:27:10 +0800 Subject: [PATCH 01/58] last, optimiser: add package ast and optimizer. --- ast/ast.go | 46 ++++++++++++++++++++++++++++++ ast/nodes.go | 63 ++++++++++++++++++++++++++++++++++++++++++ ddl/ddl_test.go | 2 +- optimizer/optimizer.go | 19 +++++++++++++ parser/parser.y | 13 +++++++-- parser/scanner.l | 5 ++-- tidb.go | 20 ++++++++++++-- 7 files changed, 158 insertions(+), 10 deletions(-) create mode 100644 ast/ast.go create mode 100644 ast/nodes.go create mode 100644 optimizer/optimizer.go diff --git a/ast/ast.go b/ast/ast.go new file mode 100644 index 0000000000..299cc70b69 --- /dev/null +++ b/ast/ast.go @@ -0,0 +1,46 @@ +// 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 + +// Node is the basic element of the AST. +type Node interface { + // Accept accepts Visitor to visit itself. + // The returned node should replace original node. + // ok returns false means stop. + 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) +} + +// Expression is a node that can be evaluated. +type Expression interface { + // Node is embeded in Expression. + Node + // IsStatic means it can be evaluated without column binding. + IsStatic() bool +} + +// Visitor visits a Node. +type Visitor interface { + // VisitEnter is called before children node has been visited. + // ok returns false means stop visiting. + VisitEnter(n Node) (node Node, ok bool) + // VisitLeave is called after chiledren node has been visited. + // ok returns false means stop visiting. + VisitLeave(n Node) (node Node, ok bool) +} diff --git a/ast/nodes.go b/ast/nodes.go new file mode 100644 index 0000000000..ec49dd76b6 --- /dev/null +++ b/ast/nodes.go @@ -0,0 +1,63 @@ +package ast + +// txtNode is the struct implements partial node interface +// can be embeded by other nodes. +type txtNode struct { + txt string +} + +// SetText implements Node interface. +func (bn *txtNode) SetText(text string) { + bn.txt = text +} + +// Text implements Node interface. +func (bn *txtNode) Text() string { + return bn.txt +} + +// VariableAssignment is a variable assignment struct. +type VariableAssignment struct { + txtNode + Name string + Value Expression + IsGlobal bool + IsSystem bool +} + +// Accept implements Node interface. +func (va *VariableAssignment) Accept(v Visitor) (Node, bool) { + node, ok := v.VisitEnter(va) + if !ok { + return node, false + } + node, ok = va.Value.Accept(v) + if !ok { + return va, false + } + va.Value = node.(Expression) + return v.VisitLeave(va) +} + +// SetStmt is the statement to set variables. +type SetStmt struct { + txtNode + // Variables is the list of variable assignment. + Variables []*VariableAssignment +} + +// Accept implements Node interface. +func (set *SetStmt) Accept(v Visitor) (Node, bool) { + node, ok := v.VisitEnter(set) + if !ok { + return node, false + } + for i, val := range set.Variables { + node, ok = val.Accept(v) + if !ok { + return set, false + } + set.Variables[i] = node.(*VariableAssignment) + } + return v.VisitLeave(set) +} \ No newline at end of file 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/optimizer/optimizer.go b/optimizer/optimizer.go new file mode 100644 index 0000000000..c4c5f1d12d --- /dev/null +++ b/optimizer/optimizer.go @@ -0,0 +1,19 @@ +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/parser.y b/parser/parser.y index 4a36de72f1..263dd9498c 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" @@ -3495,9 +3496,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 diff --git a/parser/scanner.l b/parser/scanner.l index 6cbe8f56fa..06a93b3385 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/tidb.go b/tidb.go index b9e8f7555b..1d3d7de5f5 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) { From 2fd0c7a1d38ffcc604d48c2830ad28ee47cce91a Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Mon, 12 Oct 2015 15:19:24 +0800 Subject: [PATCH 02/58] ast: address comments. --- ast/ast.go | 24 ++++++++++++++++-------- ast/nodes.go | 10 +++++----- 2 files changed, 21 insertions(+), 13 deletions(-) diff --git a/ast/ast.go b/ast/ast.go index 299cc70b69..f46506b7ae 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -15,11 +15,15 @@ // 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. type Node interface { // Accept accepts Visitor to visit itself. // The returned node should replace original node. - // ok returns false means stop. + // ok returns false to stop visiting. Accept(v Visitor) (node Node, ok bool) // Text returns the original text of the element. Text() string @@ -31,16 +35,20 @@ type Node interface { type Expression interface { // Node is embeded in Expression. Node - // IsStatic means it can be evaluated without column binding. + // IsStatic means it can be evaluated independently. IsStatic() bool + // SetType sets type to the expression. + SetType(tp *types.FieldType) + // GetType gets type of the expression. + GetType() *types.FieldType } // Visitor visits a Node. type Visitor interface { - // VisitEnter is called before children node has been visited. - // ok returns false means stop visiting. - VisitEnter(n Node) (node Node, ok bool) - // VisitLeave is called after chiledren node has been visited. - // ok returns false means stop visiting. - VisitLeave(n Node) (node Node, ok bool) + // VisitEnter is called before children nodes is visited. + // ok returns false to stop visiting. + Enter(n Node) (node 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/nodes.go b/ast/nodes.go index ec49dd76b6..7f7027e2ec 100644 --- a/ast/nodes.go +++ b/ast/nodes.go @@ -27,7 +27,7 @@ type VariableAssignment struct { // Accept implements Node interface. func (va *VariableAssignment) Accept(v Visitor) (Node, bool) { - node, ok := v.VisitEnter(va) + node, ok := v.Enter(va) if !ok { return node, false } @@ -36,7 +36,7 @@ func (va *VariableAssignment) Accept(v Visitor) (Node, bool) { return va, false } va.Value = node.(Expression) - return v.VisitLeave(va) + return v.Leave(va) } // SetStmt is the statement to set variables. @@ -48,7 +48,7 @@ type SetStmt struct { // Accept implements Node interface. func (set *SetStmt) Accept(v Visitor) (Node, bool) { - node, ok := v.VisitEnter(set) + node, ok := v.Enter(set) if !ok { return node, false } @@ -59,5 +59,5 @@ func (set *SetStmt) Accept(v Visitor) (Node, bool) { } set.Variables[i] = node.(*VariableAssignment) } - return v.VisitLeave(set) -} \ No newline at end of file + return v.Leave(set) +} From c13ab04a6a1c3ec2981bdfc6a8fd52e14769d5c7 Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Mon, 12 Oct 2015 20:28:10 +0800 Subject: [PATCH 03/58] ast: add some expression and select statement. --- ast/ast.go | 2 +- ast/expressions.go | 355 +++++++++++++++++++++++++++++++++++++++++ ast/nodes.go | 29 ++-- ast/statements.go | 81 ++++++++++ optimizer/optimizer.go | 13 ++ 5 files changed, 470 insertions(+), 10 deletions(-) create mode 100644 ast/expressions.go create mode 100644 ast/statements.go diff --git a/ast/ast.go b/ast/ast.go index f46506b7ae..ddcb37a38f 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -47,7 +47,7 @@ type Expression interface { type Visitor interface { // VisitEnter is called before children nodes is visited. // ok returns false to stop visiting. - Enter(n Node) (node Node, ok bool) + 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/expressions.go b/ast/expressions.go new file mode 100644 index 0000000000..9024c1dc98 --- /dev/null +++ b/ast/expressions.go @@ -0,0 +1,355 @@ +// 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/expression/builtin" + "github.com/pingcap/tidb/model" + "github.com/pingcap/tidb/parser/opcode" + "github.com/pingcap/tidb/util/types" + "strings" +) + +// tpNode is the struct implements partial expression interface. +// can be embeded by other nodes. +type tpNode struct { + tp *types.FieldType +} + +// SetType implements Expression interface. +func (tn *tpNode) SetType(tp *types.FieldType) { + tn.tp = tp +} + +// GetType implements Expression interface. +func (tn *tpNode) GetType() *types.FieldType { + return tn.tp +} + +// Value is the simple value expression. +type Value struct { + txtNode + tpNode + // Val is the literal value. + Val interface{} +} + +// IsStatic implements Expression. +func (val *Value) IsStatic() bool { + return true +} + +// Accept implements Node interface. +func (val *Value) Accept(v Visitor) (Node, bool) { + if !v.Enter(val) { + return val, false + } + return v.Leave(val) +} + +// Between is for "between and" or "not between and" expression. +type Between struct { + txtNode + tpNode + // Expr is the expression to be checked. + Expr Expression + // Left is the expression for minimal value in the range. + Left Expression + // Right is the expression for maximum value in the range. + Right Expression + // Not is true, the expression is "not between and". + Not bool +} + +// Accept implements Node interface. +func (b *Between) 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.(Expression) + + node, ok = b.Left.Accept(v) + if !ok { + return b, false + } + b.Left = node.(Expression) + + node, ok = b.Right.Accept(v) + if !ok { + return b, false + } + b.Right = node.(Expression) + + return v.Leave(b) +} + +// IsStatic implements the Expression IsStatic interface. +func (b *Between) IsStatic() bool { + return b.Expr.IsStatic() && b.Left.IsStatic() && b.Right.IsStatic() +} + +// BinaryOperation is for binary operation like 1 + 1, 1 - 1, etc. +type BinaryOperation struct { + txtNode + tpNode + // Op is the operator code for BinaryOperation. + Op opcode.Op + // L is the left expression in BinaryOperation. + L Expression + // R is the right expression in BinaryOperation. + R Expression +} + +// Accept implements Node interface. +func (o *BinaryOperation) 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.(Expression) + + node, ok = o.R.Accept(v) + if !ok { + return o, false + } + o.R = node.(Expression) + + return v.Leave(o) +} + +// IsStatic implements the Expression IsStatic interface. +func (o *BinaryOperation) IsStatic() bool { + return o.L.IsStatic() && o.R.IsStatic() +} + +// Call is for function expression. +type Call struct { + txtNode + tpNode + // F is the function name. + F string + // Args is the function args. + Args []Expression + // Distinct only affetcts sum, avg, count, group_concat, + // so we can ignore it in other functions + Distinct bool +} + +// Accept implements Node interface. +func (c *Call) 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.(Expression) + } + return v.Leave(c) +} + +// IsStatic implements the Expression IsStatic interface. +func (c *Call) 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 +} + +// WhenClause is the expression in Case expression for "when condition then result". +type WhenClause struct { + txtNode + tpNode + // Expr is the condition expression in WhenClause. + Expr Expression + // Result is the result expression in WhenClause. + Result Expression +} + +// 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.(Expression) + + node, ok = w.Result.Accept(v) + if !ok { + return w, false + } + w.Result = node.(Expression) + return v.Leave(w) +} + +// IsStatic implements the Expression IsStatic interface. +func (w *WhenClause) IsStatic() bool { + return w.Expr.IsStatic() && w.Result.IsStatic() +} + +// FunctionCase is the case expression. +type FunctionCase struct { + // Value is the compare value expression. + Value Expression + // WhenClauses is the condition check expression. + WhenClauses []*WhenClause + // ElseClause is the else result expression. + ElseClause Expression +} + +// Accept implements Node Accept interface. +func (f *FunctionCase) Accept(v Visitor) (Expression, bool) { + if !v.Enter(f) { + return f, false + } + node, ok := f.Value.Accept(v) + if !ok { + return f, false + } + f.Value = node.(Expression) + for i, val := range f.WhenClauses { + node, ok = val.Accept(v) + if !ok { + return f, false + } + f.WhenClauses[i] = node.(*WhenClause) + } + node, ok = f.ElseClause.Accept(v) + if !ok { + return f, false + } + f.ElseClause = node.(Expression) + return v.Leave(f) +} + +// IsStatic implements the Expression IsStatic interface. +func (f *FunctionCase) 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 +} + +// castOperatopr is the operator type for cast function. +type castFunctionType int + +const ( + // CastFunction is CAST function. + CastFunction castFunctionType = iota + 1 + // ConvertFunction is CONVERT function. + ConvertFunction + // BinaryOperator is BINARY operator. + BinaryOperator +) + +// FunctionCast 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 FunctionCast struct { + // Expr is the expression to be converted. + Expr Expression + // Tp is the conversion type. + Tp *types.FieldType + // Cast, Convert and Binary share this struct. + FunctionType castFunctionType +} + +// IsStatic implements the Expression IsStatic interface. +func (f *FunctionCast) IsStatic() bool { + return f.Expr.IsStatic() +} + +// Accept implements Node Accept interface. +func (f *FunctionCast) Accept(v Visitor) (Node, bool) { + if !v.Enter(v) { + return f, false + } + node, ok := f.Expr.Accept(v) + if !ok { + return f, false + } + f.Expr = node.(Expression) + return v.Leave(f) +} + +// SubQuery represents a sub query. +type SubQuery struct { + // Query is the query SelectNode. + Query SelectNode +} + +// CompareSubQuery 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 CompareSubQuery struct { + // L is the left expression + L Expression + // Op is the comparison opcode. + Op opcode.Op + // R is the sub query for right expression. + R SubQuery + // All is true, we should compare all records in subquery. + All bool +} + +// ColumnRef represents a column reference. +type ColumnRef struct { + txtNode + tpNode + + // Name is the referenced column name. + Name model.CIStr +} + +// Accept implements Node Accept interface. +func (cr *ColumnRef) Accept(v Visitor) (Node, bool) { + if !v.Enter(v) { + return cr, false + } + return v.Leave(cr) +} + +// IsStatic implements the Expression IsStatic interface. +func (cr *ColumnRef) IsStatic() bool { + return false +} diff --git a/ast/nodes.go b/ast/nodes.go index 7f7027e2ec..f000873b95 100644 --- a/ast/nodes.go +++ b/ast/nodes.go @@ -1,6 +1,19 @@ +// 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 -// txtNode is the struct implements partial node interface +// txtNode is the struct implements partial node interface. // can be embeded by other nodes. type txtNode struct { txt string @@ -27,11 +40,10 @@ type VariableAssignment struct { // Accept implements Node interface. func (va *VariableAssignment) Accept(v Visitor) (Node, bool) { - node, ok := v.Enter(va) - if !ok { - return node, false + if !v.Enter(va) { + return va, false } - node, ok = va.Value.Accept(v) + node, ok := va.Value.Accept(v) if !ok { return va, false } @@ -48,12 +60,11 @@ type SetStmt struct { // Accept implements Node interface. func (set *SetStmt) Accept(v Visitor) (Node, bool) { - node, ok := v.Enter(set) - if !ok { - return node, false + if !v.Enter(set) { + return set, false } for i, val := range set.Variables { - node, ok = val.Accept(v) + node, ok := val.Accept(v) if !ok { return set, false } diff --git a/ast/statements.go b/ast/statements.go new file mode 100644 index 0000000000..185ff0c4c7 --- /dev/null +++ b/ast/statements.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 + +import ( + "github.com/pingcap/tidb/table" +) + +// 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 +) + +// JoinNode represents table join. +type JoinNode struct { + txtNode + + Left Node + Right Node + Tp JoinType +} + +// TableRef represents a reference to actual table. +type TableRef struct { + // Ident is the table identifier. + Ident table.Ident +} + +// TableSource represents table source with a name. +type TableSource struct { + txtNode + + // 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 +} + +// SelectNode represents the select query node. +type SelectNode struct { + txtNode + + // Distinct represents if the select has distinct option. + Distinct bool + // Fields is the select expression list. + Fields []Expression + // From is the from clause of the query. + From JoinNode + // Where is the where clause in select statement. + Where Expression + // GroupBy is the group by expression list. + GroupBy []Expression + // Having is the having condition. + Having Expression + // OrderBy is the odering expression list. + OrderBy []Expression + // Offset is the offset value. + Offset Value + // Limit is the limit value. + Limit Value +} diff --git a/optimizer/optimizer.go b/optimizer/optimizer.go index c4c5f1d12d..36ccd06c5f 100644 --- a/optimizer/optimizer.go +++ b/optimizer/optimizer.go @@ -1,3 +1,16 @@ +// 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 ( From 716bd75c2ebbd44333258d91626573f6565361a4 Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Tue, 13 Oct 2015 13:42:17 +0800 Subject: [PATCH 04/58] ast: add all expressions --- ast/expressions.go | 624 ++++++++++++++++++++++++++++++++++++++++++++- ast/nodes.go | 74 ------ ast/statements.go | 169 +++++++++++- 3 files changed, 780 insertions(+), 87 deletions(-) delete mode 100644 ast/nodes.go diff --git a/ast/expressions.go b/ast/expressions.go index 9024c1dc98..8ace5f750f 100644 --- a/ast/expressions.go +++ b/ast/expressions.go @@ -21,6 +21,37 @@ import ( "strings" ) +var ( + _ Expression = &Value{} + _ Expression = &Between{} + _ Expression = &BinaryOperation{} + _ Expression = &Call{} + _ Expression = &WhenClause{} + _ Expression = &FunctionCase{} + _ Expression = &FunctionCast{} + _ Expression = &SubQuery{} + _ Expression = &CompareSubQuery{} + _ Expression = &ColumnRef{} + _ Expression = &FunctionConvert{} + _ Expression = &Default{} + _ Expression = &ExistsSubQuery{} + _ Expression = &Extract{} + _ Expression = &PatternIn{} + _ Expression = &IsNull{} + _ Expression = &IsTruth{} + _ Expression = &PatternLike{} + _ Expression = &ParamMarker{} + _ Expression = &PExpr{} + _ Expression = &Position{} + _ Expression = &PatternRegexp{} + _ Expression = &Row{} + _ Expression = &FunctionSubstring{} + _ Expression = &FunctionTrim{} + _ Expression = &UnaryOperation{} + _ Expression = &Values{} + _ Expression = &Variable{} +) + // tpNode is the struct implements partial expression interface. // can be embeded by other nodes. type tpNode struct { @@ -37,6 +68,12 @@ func (tn *tpNode) GetType() *types.FieldType { return tn.tp } +type nonStatic struct{} + +func (ns *nonStatic) IsStatic() bool { + return false +} + // Value is the simple value expression. type Value struct { txtNode @@ -221,6 +258,8 @@ func (w *WhenClause) IsStatic() bool { // FunctionCase is the case expression. type FunctionCase struct { + txtNode + tpNode // Value is the compare value expression. Value Expression // WhenClauses is the condition check expression. @@ -230,7 +269,7 @@ type FunctionCase struct { } // Accept implements Node Accept interface. -func (f *FunctionCase) Accept(v Visitor) (Expression, bool) { +func (f *FunctionCase) Accept(v Visitor) (Node, bool) { if !v.Enter(f) { return f, false } @@ -285,6 +324,8 @@ const ( // FunctionCast 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 FunctionCast struct { + txtNode + tpNode // Expr is the expression to be converted. Expr Expression // Tp is the conversion type. @@ -300,7 +341,7 @@ func (f *FunctionCast) IsStatic() bool { // Accept implements Node Accept interface. func (f *FunctionCast) Accept(v Visitor) (Node, bool) { - if !v.Enter(v) { + if !v.Enter(f) { return f, false } node, ok := f.Expr.Accept(v) @@ -313,8 +354,24 @@ func (f *FunctionCast) Accept(v Visitor) (Node, bool) { // SubQuery represents a sub query. type SubQuery struct { + txtNode + tpNode + nonStatic // Query is the query SelectNode. - Query SelectNode + Query *SelectNode +} + +// Accept implements Node Accept interface. +func (sq *SubQuery) 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.(*SelectNode) + return v.Leave(sq) } // CompareSubQuery is the expression for "expr cmp (select ...)". @@ -322,20 +379,42 @@ type SubQuery struct { // 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 CompareSubQuery struct { + txtNode + tpNode + nonStatic // L is the left expression L Expression // Op is the comparison opcode. Op opcode.Op // R is the sub query for right expression. - R SubQuery + R *SubQuery // All is true, we should compare all records in subquery. All bool } +// Accept implements Node Accept interface. +func (cs *CompareSubQuery) 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.(Expression) + node, ok = cs.R.Accept(v) + if !ok { + return cs, false + } + cs.R = node.(*SubQuery) + return v.Leave(cs) +} + // ColumnRef represents a column reference. type ColumnRef struct { txtNode tpNode + nonStatic // Name is the referenced column name. Name model.CIStr @@ -343,13 +422,542 @@ type ColumnRef struct { // Accept implements Node Accept interface. func (cr *ColumnRef) Accept(v Visitor) (Node, bool) { - if !v.Enter(v) { + if !v.Enter(cr) { return cr, false } return v.Leave(cr) } -// IsStatic implements the Expression IsStatic interface. -func (cr *ColumnRef) IsStatic() bool { - return false +// FunctionConvert 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 FunctionConvert struct { + txtNode + tpNode + // Expr is the expression to be converted. + Expr Expression + // Charset is the target character set to convert. + Charset string +} + +// IsStatic implements the Expression IsStatic interface. +func (f *FunctionConvert) IsStatic() bool { + return f.Expr.IsStatic() +} + +// Accept implements Node Accept interface. +func (f *FunctionConvert) 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.(Expression) + return v.Leave(f) +} + +// Default is the default expression using default value for a column. +type Default struct { + txtNode + tpNode + nonStatic + // Name is the column name. + Name string +} + +// Accept implements Node Accept interface. +func (d *Default) Accept(v Visitor) (Node, bool) { + if !v.Enter(d) { + return d, false + } + return v.Leave(d) +} + +// ExistsSubQuery is the expression for "exists (select ...)". +// https://dev.mysql.com/doc/refman/5.7/en/exists-and-not-exists-subqueries.html +type ExistsSubQuery struct { + txtNode + tpNode + nonStatic + // Sel is the sub query. + Sel *SubQuery +} + +// Accept implements Node Accept interface. +func (es *ExistsSubQuery) 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.(*SubQuery) + return v.Leave(es) +} + +// Extract is for time extract function. +// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_extract +type Extract struct { + txtNode + tpNode + + Unit string + Date Expression +} + +// Accept implements Node Accept interface. +func (ex *Extract) 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.(Expression) + return v.Leave(ex) +} + +// IsStatic implements the Expression IsStatic interface. +func (ex *Extract) IsStatic() bool { + return ex.Date.IsStatic() +} + +// PatternIn is the expression for in operator, like "expr in (1, 2, 3)" or "expr in (select c from t)". +type PatternIn struct { + txtNode + tpNode + nonStatic + // Expr is the value expression to be compared. + Expr Expression + // List is the list expression in compare list. + List []Expression + // Not is true, the expression is "not in". + Not bool + // Sel is the sub query. + Sel *SubQuery +} + +// Accept implements Node Accept interface. +func (pi *PatternIn) 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.(Expression) + for i, val := range pi.List { + node, ok = val.Accept(v) + if !ok { + return pi, false + } + pi.List[i] = node.(Expression) + } + if pi.Sel != nil { + node, ok = pi.Sel.Accept(v) + if !ok { + return pi, false + } + pi.Sel = node.(*SubQuery) + } + return v.Leave(pi) +} + +// IsNull is the expression for null check. +type IsNull struct { + txtNode + tpNode + // Expr is the expression to be checked. + Expr Expression + // Not is true, the expression is "is not null". + Not bool +} + +// Accept implements Node Accept interface. +func (is *IsNull) 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.(Expression) + return v.Leave(is) +} + +// IsStatic implements the Expression IsStatic interface. +func (is *IsNull) IsStatic() bool { + return is.Expr.IsStatic() +} + +// IsTruth is the expression for true/false check. +type IsTruth struct { + txtNode + tpNode + // Expr is the expression to be checked. + Expr Expression + // 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 *IsTruth) 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.(Expression) + return v.Leave(is) +} + +// IsStatic implements the Expression IsStatic interface. +func (is *IsTruth) IsStatic() bool { + return is.Expr.IsStatic() +} + +// PatternLike is the expression for like operator, e.g, expr like "%123%" +type PatternLike struct { + txtNode + tpNode + // Expr is the expression to be checked. + Expr Expression + // Pattern is the like expression. + Pattern Expression + // Not is true, the expression is "not like". + Not bool +} + +// Accept implements Node Accept interface. +func (pl *PatternLike) 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.(Expression) + node, ok = pl.Pattern.Accept(v) + if !ok { + return pl, false + } + pl.Pattern = node.(Expression) + return v.Leave(pl) +} + +// IsStatic implements the Expression IsStatic interface. +func (pl *PatternLike) IsStatic() bool { + return pl.Expr.IsStatic() && pl.Pattern.IsStatic() +} + +// ParamMarker expresion holds a place for another expression. +// Used in parsing prepare statement. +type ParamMarker struct { + txtNode + tpNode + // Expr is the expression to be evaluated in this place holder. + Expr Expression +} + +// Accept implements Node Accept interface. +func (pm *ParamMarker) Accept(v Visitor) (Node, bool) { + if !v.Enter(pm) { + return pm, false + } + node, ok := pm.Expr.Accept(v) + if !ok { + return pm, false + } + pm.Expr = node.(Expression) + return v.Leave(pm) +} + +// IsStatic implements the Expression IsStatic interface. +func (pm *ParamMarker) IsStatic() bool { + return pm.Expr.IsStatic() +} + +// PExpr is the parenthesis expression. +type PExpr struct { + txtNode + tpNode + // Expr is the expression in parenthesis. + Expr Expression +} + +// Accept implements Node Accept interface. +func (p *PExpr) 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.(Expression) + return v.Leave(p) +} + +// IsStatic implements the Expression IsStatic interface. +func (p *PExpr) IsStatic() bool { + return p.Expr.IsStatic() +} + +// Position 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 Position struct { + txtNode + tpNode + // 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 Expression IsStatic interface. +func (p *Position) IsStatic() bool { + return true +} + +// Accept implements Node Accept interface. +func (p *Position) Accept(v Visitor) (Node, bool) { + if !v.Enter(p) { + return p, false + } + return v.Leave(p) +} + +// PatternRegexp is the pattern expression for pattern match. +type PatternRegexp struct { + txtNode + tpNode + // Expr is the expression to be checked. + Expr Expression + // Pattern is the expression for pattern. + Pattern Expression + // Not is true, the expression is "not rlike", + Not bool +} + +// Accept implements Node Accept interface. +func (p *PatternRegexp) 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.(Expression) + node, ok = p.Pattern.Accept(v) + if !ok { + return p, false + } + p.Pattern = node.(Expression) + return v.Leave(p) +} + +// IsStatic implements the Expression IsStatic interface. +func (p *PatternRegexp) IsStatic() bool { + return p.Expr.IsStatic() && p.Pattern.IsStatic() +} + +// Row is the expression for row constructor. +// See https://dev.mysql.com/doc/refman/5.7/en/row-subqueries.html +type Row struct { + txtNode + tpNode + + Values []Expression +} + +// Accept implements Node Accept interface. +func (r *Row) 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.(Expression) + } + return v.Leave(r) +} + +// IsStatic implements the Expression IsStatic interface. +func (r *Row) IsStatic() bool { + for _, v := range r.Values { + if !v.IsStatic() { + return false + } + } + return true +} + +// FunctionSubstring returns the substring as specified. +// See: https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_substring +type FunctionSubstring struct { + txtNode + tpNode + + StrExpr Expression + Pos Expression + Len Expression +} + +// Accept implements Node Accept interface. +func (fs *FunctionSubstring) Accept(v Visitor) (Node, bool) { + if !v.Enter(fs) { + return fs, false + } + node, ok := fs.StrExpr.Accept(v) + if !ok { + return fs, false + } + fs.StrExpr = node.(Expression) + node, ok = fs.Pos.Accept(v) + if !ok { + return fs, false + } + fs.Pos = node.(Expression) + node, ok = fs.Len.Accept(v) + if !ok { + return fs, false + } + fs.Len = node.(Expression) + return v.Leave(fs) +} + +// IsStatic implements the Expression IsStatic interface. +func (fs *FunctionSubstring) IsStatic() bool { + return fs.StrExpr.IsStatic() && fs.Pos.IsStatic() && fs.Len.IsStatic() +} + +const ( + // TrimBothDefault trims from both direction by default. + TrimBothDefault = iota + // TrimBoth trims from both direction with explicit notation. + TrimBoth + // TrimLeading trims from left. + TrimLeading + // TrimTrailing trims from right. + TrimTrailing +) + +// FunctionTrim remove leading/trailing/both remstr. +// See: https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim +type FunctionTrim struct { + txtNode + tpNode + + Str Expression + RemStr Expression + Direction int +} + +// Accept implements Node Accept interface. +func (ft *FunctionTrim) Accept(v Visitor) (Node, bool) { + if !v.Enter(ft) { + return ft, false + } + node, ok := ft.Str.Accept(v) + if !ok { + return ft, false + } + ft.Str = node.(Expression) + node, ok = ft.RemStr.Accept(v) + if !ok { + return ft, false + } + ft.RemStr = node.(Expression) + return v.Leave(ft) +} + +// IsStatic implements the Expression IsStatic interface. +func (ft *FunctionTrim) IsStatic() bool { + return ft.Str.IsStatic() && ft.RemStr.IsStatic() +} + +// UnaryOperation is the expression for unary operator. +type UnaryOperation struct { + txtNode + tpNode + // Op is the operator opcode. + Op opcode.Op + // V is the unary expression. + V Expression +} + +// Accept implements Node Accept interface. +func (u *UnaryOperation) 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.(Expression) + return v.Leave(u) +} + +// IsStatic implements the Expression IsStatic interface. +func (u *UnaryOperation) IsStatic() bool { + return u.V.IsStatic() +} + +// Values is the expression used in INSERT VALUES +type Values struct { + txtNode + tpNode + nonStatic + // model.CIStr is column name. + Column *ColumnRef +} + +// Accept implements Node Accept interface. +func (va *Values) 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.(*ColumnRef) + return v.Leave(va) +} + +// Variable is the expression for variable. +type Variable struct { + txtNode + tpNode + nonStatic + // 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 *Variable) Accept(v Visitor) (Node, bool) { + if !v.Enter(va) { + return va, false + } + return v.Leave(va) } diff --git a/ast/nodes.go b/ast/nodes.go deleted file mode 100644 index f000873b95..0000000000 --- a/ast/nodes.go +++ /dev/null @@ -1,74 +0,0 @@ -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package ast - -// txtNode is the struct implements partial node interface. -// can be embeded by other nodes. -type txtNode struct { - txt string -} - -// SetText implements Node interface. -func (bn *txtNode) SetText(text string) { - bn.txt = text -} - -// Text implements Node interface. -func (bn *txtNode) Text() string { - return bn.txt -} - -// VariableAssignment is a variable assignment struct. -type VariableAssignment struct { - txtNode - Name string - Value Expression - 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.(Expression) - return v.Leave(va) -} - -// SetStmt is the statement to set variables. -type SetStmt struct { - txtNode - // Variables is the list of variable assignment. - Variables []*VariableAssignment -} - -// Accept implements Node 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/ast/statements.go b/ast/statements.go index 185ff0c4c7..26c0dbc82b 100644 --- a/ast/statements.go +++ b/ast/statements.go @@ -17,6 +17,29 @@ import ( "github.com/pingcap/tidb/table" ) +var ( + _ Node = &SelectNode{} + _ Node = &JoinNode{} + _ Node = &TableRef{} + _ Node = &TableSource{} +) + +// txtNode is the struct implements partial node interface. +// can be embeded by other nodes. +type txtNode struct { + txt string +} + +// SetText implements Node interface. +func (bn *txtNode) SetText(text string) { + bn.txt = text +} + +// Text implements Node interface. +func (bn *txtNode) Text() string { + return bn.txt +} + // JoinType is join type, including cross/left/right/full. type JoinType int @@ -33,17 +56,47 @@ const ( type JoinNode struct { txtNode - Left Node + // Left table can be TableSource or JoinNode. + Left Node + // Right table can be TableSource or JoinNode or nil. Right Node - Tp JoinType + // Tp represents join type. + Tp JoinType +} + +// Accept implements Node Accept interface. +func (jn *JoinNode) Accept(v Visitor) (Node, bool) { + if !v.Enter(jn) { + return jn, false + } + node, ok := jn.Left.Accept(v) + if !ok { + return jn, false + } + jn.Left = node + node, ok = jn.Right.Accept(v) + if !ok { + return jn, false + } + jn.Right = node + return v.Leave(jn) } // TableRef represents a reference to actual table. type TableRef struct { + txtNode // Ident is the table identifier. Ident table.Ident } +// 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 { txtNode @@ -56,6 +109,19 @@ type TableSource struct { 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) +} + // SelectNode represents the select query node. type SelectNode struct { txtNode @@ -65,7 +131,7 @@ type SelectNode struct { // Fields is the select expression list. Fields []Expression // From is the from clause of the query. - From JoinNode + From *JoinNode // Where is the where clause in select statement. Where Expression // GroupBy is the group by expression list. @@ -75,7 +141,100 @@ type SelectNode struct { // OrderBy is the odering expression list. OrderBy []Expression // Offset is the offset value. - Offset Value + Offset int // Limit is the limit value. - Limit Value + Limit int +} + +// Accept implements Node Accept interface. +func (sn *SelectNode) 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.(Expression) + } + node, ok := sn.From.Accept(v) + if !ok { + return sn, false + } + sn.From = node.(*JoinNode) + + node, ok = sn.Where.Accept(v) + if !ok { + return sn, false + } + sn.Where = node.(Expression) + + for i, val := range sn.GroupBy { + node, ok = val.Accept(v) + if !ok { + return sn, false + } + sn.GroupBy[i] = node.(Expression) + } + + node, ok = sn.Having.Accept(v) + if !ok { + return sn, false + } + sn.Having = node.(Expression) + + for i, val := range sn.OrderBy { + node, ok = val.Accept(v) + if !ok { + return sn, false + } + sn.OrderBy[i] = node.(Expression) + } + + return v.Leave(sn) +} + +// VariableAssignment is a variable assignment struct. +type VariableAssignment struct { + txtNode + Name string + Value Expression + 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.(Expression) + return v.Leave(va) +} + +// SetStmt is the statement to set variables. +type SetStmt struct { + txtNode + // Variables is the list of variable assignment. + Variables []*VariableAssignment +} + +// Accept implements Node 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) } From d50b6c5fcc8b72e014cf41004bc0793ec596ffda Mon Sep 17 00:00:00 2001 From: shenli Date: Tue, 13 Oct 2015 18:11:11 +0800 Subject: [PATCH 05/58] *: Fix hibernate bug about col.Flen/Decimal --- field/result_field.go | 9 --------- mysqldef/util.go | 20 +++++++++++++++++++- mysqldef/util_test.go | 4 +++- parser/coldef/col_def.go | 5 ++++- plan/plans/info.go | 10 ++++++++-- tidb-server/server/conn.go | 1 + tidb-server/server/driver_tidb.go | 12 ++++++++++-- tidb_test.go | 2 +- util/types/etc.go | 8 ++++---- util/types/etc_test.go | 16 ++++++++-------- 10 files changed, 58 insertions(+), 29 deletions(-) diff --git a/field/result_field.go b/field/result_field.go index 9e3803b01a..c6847bc3db 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 { 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/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/plan/plans/info.go b/plan/plans/info.go index 2779dfbe1e..3a2b214314 100644 --- a/plan/plans/info.go +++ b/plan/plans/info.go @@ -361,8 +361,14 @@ 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 := types.TypeToStr(col.Tp, col.Charset == charset.CharsetBin) + if col.Decimal == types.UnspecifiedLength { + if colLen != types.UnspecifiedLength { + columnType = fmt.Sprintf("%s(%d)", columnType, colLen) + } + } else { + columnType = fmt.Sprintf("%s(%d, %d)", columnType, colLen, col.Decimal) + } columnDesc := column.NewColDesc(&column.Col{ColumnInfo: *col}) var columnDefault interface{} if columnDesc.DefaultValue != nil { diff --git a/tidb-server/server/conn.go b/tidb-server/server/conn.go index 26f290b4bd..5f19f589fb 100644 --- a/tidb-server/server/conn.go +++ b/tidb-server/server/conn.go @@ -433,6 +433,7 @@ func (cc *clientConn) writeResultset(rs ResultSet, binary bool) error { var rowData []byte rowData, err = dumpRowValuesBinary(cc.alloc, columns, row) if err != nil { + fmt.Println("[conn]", err) return errors.Trace(err) } data = append(data, rowData...) diff --git a/tidb-server/server/driver_tidb.go b/tidb-server/server/driver_tidb.go index 1ea32908fa..f536ca92ec 100644 --- a/tidb-server/server/driver_tidb.go +++ b/tidb-server/server/driver_tidb.go @@ -272,8 +272,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 == -1 { + ci.ColumnLength = 0 + } else { + ci.ColumnLength = uint32(fld.Flen) + } + if fld.Decimal == -1 { + ci.Decimal = 0 + } else { + ci.Decimal = uint8(fld.Decimal) + } ci.Type = uint8(fld.Tp) return } diff --git a/tidb_test.go b/tidb_test.go index b14b5fdbad..71e957420c 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -888,7 +888,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() diff --git a/util/types/etc.go b/util/types/etc.go index 7387e25515..b2e7cde4da 100644 --- a/util/types/etc.go +++ b/util/types/etc.go @@ -89,22 +89,22 @@ func TypeStr(tp byte) (r string) { func TypeToStr(tp byte, binary bool) string { switch tp { case mysql.TypeBlob: - if binary { + if !binary { return "text" } return "blob" case mysql.TypeLongBlob: - if binary { + if !binary { return "longtext" } return "longblob" case mysql.TypeTinyBlob: - if binary { + if !binary { return "tinytext" } return "tinyblob" case mysql.TypeMediumBlob: - if binary { + if !binary { return "mediumtext" } return "mediumblob" diff --git a/util/types/etc_test.go b/util/types/etc_test.go index 6212535605..b1af52f00d 100644 --- a/util/types/etc_test.go +++ b/util/types/etc_test.go @@ -68,17 +68,17 @@ 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.TypeBlob, false, "text") + testTypeToStr(c, mysql.TypeLongBlob, false, "longtext") + testTypeToStr(c, mysql.TypeTinyBlob, false, "tinytext") + testTypeToStr(c, mysql.TypeMediumBlob, false, "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.TypeBlob, true, "blob") + testTypeToStr(c, mysql.TypeLongBlob, true, "longblob") + testTypeToStr(c, mysql.TypeTinyBlob, true, "tinyblob") + testTypeToStr(c, mysql.TypeMediumBlob, true, "mediumblob") testTypeToStr(c, mysql.TypeVarchar, false, "varchar") testTypeToStr(c, mysql.TypeString, false, "char") testTypeToStr(c, mysql.TypeShort, true, "smallint") From 447e2c022e12d7fc4c391c102ae151c84694e28d Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Tue, 13 Oct 2015 20:40:20 +0800 Subject: [PATCH 06/58] ast: add statements --- ast/ddl.go | 215 ++++++++++++++++++++++++ ast/dml.go | 408 ++++++++++++++++++++++++++++++++++++++++++++++ ast/misc.go | 152 +++++++++++++++++ ast/statements.go | 240 --------------------------- 4 files changed, 775 insertions(+), 240 deletions(-) create mode 100644 ast/ddl.go create mode 100644 ast/dml.go create mode 100644 ast/misc.go delete mode 100644 ast/statements.go diff --git a/ast/ddl.go b/ast/ddl.go new file mode 100644 index 0000000000..0ab0bf82c3 --- /dev/null +++ b/ast/ddl.go @@ -0,0 +1,215 @@ +package ast + +import ( + "github.com/pingcap/tidb/util/types" +) + +var ( + _ Node = &CreateDatabaseStmt{} + _ Node = &DropDatabaseStmt{} + _ Node = &IndexColName{} + _ Node = &ReferenceDef{} + _ Node = &ConstraintOpt{} + _ Node = &TableConstraint{} + _ Node = &ColumnDef{} + _ Node = &CreateTableOption{} + _ Node = &CreateTableStmt{} + _ Node = &DropTableStmt{} + _ Node = &CreateIndexStmt{} + _ Node = &DropTableStmt{} + _ Node = &AlterTableOpt{} + _ Node = &ColumnPosition{} + _ Node = &AlterSpecification{} + _ Node = &AlterTableStmt{} +) + +// 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 { + txtNode + + IfNotExists bool + Name string + Opt *CharsetOpt +} + +// 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 { + txtNode + + IfExists bool + Name string +} + +// IndexColName is used for parsing index column name from SQL. +type IndexColName struct { + Column *ColumnRef + Length int +} + +// ReferenceDef is used for parsing foreign key reference option from SQL. +// See: http://dev.mysql.com/doc/refman/5.7/en/create-table-foreign-keys.html +type ReferenceDef struct { + TableIdent *TableRef + IndexColNames []*IndexColName +} + +// Constraints. +const ( + ConstrNoConstr = iota + ConstrPrimaryKey + ConstrForeignKey + ConstrNotNull + ConstrAutoIncrement + ConstrDefaultValue + ConstrUniq + ConstrIndex + ConstrUniqIndex + ConstrKey + ConstrUniqKey + ConstrNull + ConstrOnUpdate + ConstrFulltext + ConstrComment +) + +// ConstraintOpt is used for parsing column constraint info from SQL. +type ConstraintOpt struct { + Tp int + Bvalue bool + Evalue Expression +} + +// Table Options. +const ( + TblOptNone = iota + TblOptEngine + TblOptCharset + TblOptCollate + TblOptAutoIncrement + TblOptComment + TblOptAvgRowLength + TblOptCheckSum + TblOptCompression + TblOptConnection + TblOptPassword + TblOptKeyBlockSize + TblOptMaxRows + TblOptMinRows +) + +// TableConstraint is constraint for table definition. +type TableConstraint struct { + Tp int + ConstrName string + + // Used for PRIMARY KEY, UNIQUE, ...... + Keys []*IndexColName + + // Used for foreign key. + Refer *ReferenceDef +} + +// ColumnDef is used for parsing column definition from SQL. +type ColumnDef struct { + Name string + Tp *types.FieldType + Constraints []*ConstraintOpt +} + +// CreateTableOption is the collection of table options. +type CreateTableOption struct { + Engine string + Charset string + Collate string + AutoIncrement uint64 +} + +// CreateTableStmt is a statement to create a table. +// See: https://dev.mysql.com/doc/refman/5.7/en/create-table.html +type CreateTableStmt struct { + txtNode + + IfNotExists bool + Ident TableIdent + Cols []*ColumnDef + Constraints []*TableConstraint + Opt *CreateTableOption +} + +// 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 { + txtNode + + IfExists bool + TableRefs []*TableRef +} + +// CreateIndexStmt is a statement to create an index. +// See: https://dev.mysql.com/doc/refman/5.7/en/create-index.html +type CreateIndexStmt struct { + txtNode + + IndexName string + Table *TableRef + Unique bool + IndexColNames []*IndexColName +} + +// DropIndexStmt is a statement to drop the index. +// See: https://dev.mysql.com/doc/refman/5.7/en/drop-index.html +type DropIndexStmt struct { + IfExists bool + IndexName string + + Text string +} + +// AlterTableOpt is used for parsing table option from SQL. +type AlterTableOpt struct { + Tp int + StrValue string + UintValue uint64 +} + +// ColumnPosition Types +const ( + ColumnPositionNone int = iota + ColumnPositionFirst + ColumnPositionAfter +) + +// ColumnPosition represent the position of the newly added column +type ColumnPosition struct { + // ColumnPositionNone | ColumnPositionFirst | ColumnPositionAfter + Type int + // RelativeColumn is the column the newly added column after if type is ColumnPositionAfter + RelativeColumn *ColumnRef +} + +// AlterSpecification alter table specification +type AlterSpecification struct { + Action int + Name string + Constraint *TableConstraint + TableOpts []*AlterTableOpt + Column *ColumnDef + Position *ColumnPosition +} + +// 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 { + txtNode + + Ident TableIdent + Specs []*AlterSpecification +} diff --git a/ast/dml.go b/ast/dml.go new file mode 100644 index 0000000000..aa30c22fec --- /dev/null +++ b/ast/dml.go @@ -0,0 +1,408 @@ +// 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 ( + _ Node = &JoinNode{} + _ Node = &TableRef{} + _ Node = &TableSource{} + _ Node = &SelectNode{} + _ Node = &Assignment{} + _ Node = &InsertIntoStmt{} + _ Node = &DeleteStmt{} + _ Node = &UpdateStmt{} + _ Node = &TruncateTableStmt{} + _ Node = &UnionStmt{} +) + +// txtNode is the struct implements partial node interface. +// can be embeded by other nodes. +type txtNode struct { + txt string +} + +// SetText implements Node interface. +func (bn *txtNode) SetText(text string) { + bn.txt = text +} + +// Text implements Node interface. +func (bn *txtNode) Text() string { + return bn.txt +} + +// 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 +) + +// JoinNode represents table join. +type JoinNode struct { + txtNode + + // 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 (jn *JoinNode) Accept(v Visitor) (Node, bool) { + if !v.Enter(jn) { + return jn, false + } + node, ok := jn.Left.Accept(v) + if !ok { + return jn, false + } + jn.Left = node + node, ok = jn.Right.Accept(v) + if !ok { + return jn, false + } + jn.Right = node + return v.Leave(jn) +} + +type TableIdent struct { + Schema model.CIStr + Name model.CIStr +} + +// TableRef represents a reference to actual table. +type TableRef struct { + txtNode + + Ident TableIdent +} + +// 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 { + txtNode + + // 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) +} + +// LockType is select lock type. +type LockType int + +// Select Lock Type. +const ( + SelectLockNone LockType = iota + SelectLockForUpdate + SelectLockInShareMode +) + +// SelectNode represents the select query node. +type SelectNode struct { + txtNode + + // Distinct represents if the select has distinct option. + Distinct bool + // Fields is the select expression list. + Fields []Expression + // From is the from clause of the query. + From *JoinNode + // Where is the where clause in select statement. + Where Expression + // GroupBy is the group by expression list. + GroupBy []Expression + // Having is the having condition. + Having Expression + // OrderBy is the odering expression list. + OrderBy []Expression + // Offset is the offset value. + Offset int + // Limit is the limit value. + Limit int + // Lock is the lock type + LockTp LockType +} + +// Accept implements Node Accept interface. +func (sn *SelectNode) 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.(Expression) + } + node, ok := sn.From.Accept(v) + if !ok { + return sn, false + } + sn.From = node.(*JoinNode) + + node, ok = sn.Where.Accept(v) + if !ok { + return sn, false + } + sn.Where = node.(Expression) + + for i, val := range sn.GroupBy { + node, ok = val.Accept(v) + if !ok { + return sn, false + } + sn.GroupBy[i] = node.(Expression) + } + + node, ok = sn.Having.Accept(v) + if !ok { + return sn, false + } + sn.Having = node.(Expression) + + for i, val := range sn.OrderBy { + node, ok = val.Accept(v) + if !ok { + return sn, false + } + sn.OrderBy[i] = node.(Expression) + } + + return v.Leave(sn) +} + +// Assignment is the expression for assignment, like a = 1. +type Assignment struct { + txtNode + // Column is the column reference to be assigned. + Column *ColumnRef + // Expr is the expression assigning to ColName. + Expr Expression +} + +// 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.(*ColumnRef) + node, ok = as.Expr.Accept(v) + if !ok { + return as, false + } + as.Expr = node.(Expression) + return v.Leave(as) +} + +// InsertIntoStmt is a statement to insert new rows into an existing table. +// See: https://dev.mysql.com/doc/refman/5.7/en/insert.html +type InsertIntoStmt struct { + txtNode + + Columns []*ColumnRef + Lists [][]Expression + Table *TableRef + Setlist []*Assignment + Priority int + OnDuplicate []*Assignment +} + +// Accept implements Node Accept interface. +func (in *InsertIntoStmt) 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.(*ColumnRef) + } + 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.(Expression) + } + } + 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 { + txtNode + + Tables []*TableRef + Where Expression + Order []Expression + 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) + } + + node, ok := de.Where.Accept(v) + if !ok { + return de, false + } + de.Where = node.(Expression) + for i, val := range de.Order { + node, ok = val.Accept(v) + if !ok { + return de, false + } + de.Order[i] = node.(Expression) + } + 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 { + txtNode + + TableRefs *JoinNode + List []*Assignment + Where Expression + Order []Expression + 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.(*JoinNode) + for i, val := range up.List { + node, ok = val.Accept(v) + if !ok { + return up, false + } + up.List[i] = node.(*Assignment) + } + node, ok = up.Where.Accept(v) + if !ok { + return up, false + } + up.Where = node.(Expression) + for i, val := range up.Order { + node, ok = val.Accept(v) + if !ok { + return up, false + } + up.Order[i] = node.(Expression) + } + return v.Leave(up) +} + +// 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 { + txtNode + + Table *TableRef +} + +// UnionStmt is a statement to combine results from multiple SelectStmts. +// See: https://dev.mysql.com/doc/refman/5.7/en/union.html +type UnionStmt struct { + txtNode + + Distincts []bool + Selects []*SelectNode + Limit int + Offset int + OrderBy []Expression +} diff --git a/ast/misc.go b/ast/misc.go new file mode 100644 index 0000000000..a6d29a3b06 --- /dev/null +++ b/ast/misc.go @@ -0,0 +1,152 @@ +package ast + +var ( + _ Node = &ExplainStmt{} + _ Node = &PreparedStmt{} + _ Node = &DeallocateStmt{} + _ Node = &ExecuteStmt{} + _ Node = &BeginStmt{} + _ Node = &CommitStmt{} + _ Node = &RollbackStmt{} + _ Node = &VariableAssignment{} + _ Node = &SetStmt{} +) + +// 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 { + txtNode + + S Node +} + +// PreparedStmt 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 PreparedStmt struct { + txtNode + + InPrepare bool // true for prepare mode, false for use mode + Name string + ID uint32 // For binary protocol, there is no Name but only ID + SQLVar *Variable + SQLStmt Node // The parsed statement from sql text with placeholder + Params []*ParamMarker +} + +// DeallocateStmt is a statement to release PreparedStmt. +// See: https://dev.mysql.com/doc/refman/5.7/en/deallocate-prepare.html +type DeallocateStmt struct { + txtNode + + Name string + ID uint32 // For binary protocol, there is no Name but only ID. +} + +// ExecuteStmt is a statement to execute PreparedStmt. +// See: https://dev.mysql.com/doc/refman/5.7/en/execute.html +type ExecuteStmt struct { + txtNode + + Name string + ID uint32 // For binary protocol, there is no Name but only ID + UsingVars []Expression +} + +// 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 { + txtNode + + Target int // Databases/Tables/Columns/.... + DBName string + Table *TableRef // Used for showing columns. + ColumnName string // Used for `desc table column`. + Flag int // Some flag parsed from sql, such as FULL. + Full bool + + // Used by show variables + GlobalScope bool + Pattern *PatternLike + Where Expression +} + +// BeginStmt is a statement to start a new transaction. +// See: https://dev.mysql.com/doc/refman/5.7/en/commit.html +type BeginStmt struct { + txtNode +} + +// CommitStmt is a statement to commit the current transaction. +// See: https://dev.mysql.com/doc/refman/5.7/en/commit.html +type CommitStmt struct { + txtNode +} + +// 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 { + txtNode +} + +// 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 { + txtNode + + DBName string +} + +// VariableAssignment is a variable assignment struct. +type VariableAssignment struct { + txtNode + Name string + Value Expression + 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.(Expression) + return v.Leave(va) +} + +// SetStmt is the statement to set variables. +type SetStmt struct { + txtNode + // 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/ast/statements.go b/ast/statements.go deleted file mode 100644 index 26c0dbc82b..0000000000 --- a/ast/statements.go +++ /dev/null @@ -1,240 +0,0 @@ -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package ast - -import ( - "github.com/pingcap/tidb/table" -) - -var ( - _ Node = &SelectNode{} - _ Node = &JoinNode{} - _ Node = &TableRef{} - _ Node = &TableSource{} -) - -// txtNode is the struct implements partial node interface. -// can be embeded by other nodes. -type txtNode struct { - txt string -} - -// SetText implements Node interface. -func (bn *txtNode) SetText(text string) { - bn.txt = text -} - -// Text implements Node interface. -func (bn *txtNode) Text() string { - return bn.txt -} - -// 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 -) - -// JoinNode represents table join. -type JoinNode struct { - txtNode - - // 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 (jn *JoinNode) Accept(v Visitor) (Node, bool) { - if !v.Enter(jn) { - return jn, false - } - node, ok := jn.Left.Accept(v) - if !ok { - return jn, false - } - jn.Left = node - node, ok = jn.Right.Accept(v) - if !ok { - return jn, false - } - jn.Right = node - return v.Leave(jn) -} - -// TableRef represents a reference to actual table. -type TableRef struct { - txtNode - // Ident is the table identifier. - Ident table.Ident -} - -// 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 { - txtNode - - // 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) -} - -// SelectNode represents the select query node. -type SelectNode struct { - txtNode - - // Distinct represents if the select has distinct option. - Distinct bool - // Fields is the select expression list. - Fields []Expression - // From is the from clause of the query. - From *JoinNode - // Where is the where clause in select statement. - Where Expression - // GroupBy is the group by expression list. - GroupBy []Expression - // Having is the having condition. - Having Expression - // OrderBy is the odering expression list. - OrderBy []Expression - // Offset is the offset value. - Offset int - // Limit is the limit value. - Limit int -} - -// Accept implements Node Accept interface. -func (sn *SelectNode) 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.(Expression) - } - node, ok := sn.From.Accept(v) - if !ok { - return sn, false - } - sn.From = node.(*JoinNode) - - node, ok = sn.Where.Accept(v) - if !ok { - return sn, false - } - sn.Where = node.(Expression) - - for i, val := range sn.GroupBy { - node, ok = val.Accept(v) - if !ok { - return sn, false - } - sn.GroupBy[i] = node.(Expression) - } - - node, ok = sn.Having.Accept(v) - if !ok { - return sn, false - } - sn.Having = node.(Expression) - - for i, val := range sn.OrderBy { - node, ok = val.Accept(v) - if !ok { - return sn, false - } - sn.OrderBy[i] = node.(Expression) - } - - return v.Leave(sn) -} - -// VariableAssignment is a variable assignment struct. -type VariableAssignment struct { - txtNode - Name string - Value Expression - 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.(Expression) - return v.Leave(va) -} - -// SetStmt is the statement to set variables. -type SetStmt struct { - txtNode - // Variables is the list of variable assignment. - Variables []*VariableAssignment -} - -// Accept implements Node 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) -} From 21042911b595780cdd6354de36c258474c52f5bd Mon Sep 17 00:00:00 2001 From: siddontang Date: Tue, 13 Oct 2015 21:38:14 +0800 Subject: [PATCH 07/58] stmts: refactor update/insert/delete qualified and as handle. --- stmt/stmts/delete.go | 16 +++-- stmt/stmts/delete_test.go | 3 + stmt/stmts/insert.go | 21 ++++-- stmt/stmts/insert_test.go | 6 ++ stmt/stmts/update.go | 145 ++++++++++++++++++-------------------- stmt/stmts/update_test.go | 47 ++++++++++++ 6 files changed, 152 insertions(+), 86 deletions(-) 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..20f20b93a5 100644 --- a/stmt/stmts/insert.go +++ b/stmt/stmts/insert.go @@ -232,6 +232,12 @@ func (s *InsertIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) } insertValueCount := len(s.Lists[0]) + toUpdateColumns, err0 := checkUpdateColumns(s.OnDuplicate, nil, 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 +300,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) } 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/update.go b/stmt/stmts/update.go index ce0b698162..dde51aa787 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,52 @@ 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 checkUpdateColumns(assignList []expression.Assignment, fields []*field.ResultField, t table.Table) (map[int]expression.Assignment, error) { + m := make(map[int]expression.Assignment, len(assignList)) + + for _, v := range assignList { + if fields != nil { + 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 + } else { + c, err := findColumnByName(t, field.JoinQualifiedName("", v.TableName, v.ColName)) + if err != nil { + return nil, errors.Trace(err) + } + m[c.Offset] = 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 +138,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 +201,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 +252,16 @@ 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 := checkUpdateColumns(s.List, fs, nil) + if err0 != nil { + return nil, errors.Trace(err0) } + + m := map[interface{}]interface{}{} for { row, err1 := p.Next(ctx) if err1 != nil { @@ -274,15 +276,19 @@ func (s *UpdateStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) { return nil, nil } // 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 +299,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..34e066202a 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,48 @@ 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) } + +// See https://github.com/pingcap/tidb/issues/369 +func (s *testStmtSuite) TestIssue369(c *C) { + testSQL := `drop table if exists users, foobar;` + mustExec(c, s.testDB, testSQL) + + testSQL = `CREATE TABLE users ( + id INTEGER NOT NULL AUTO_INCREMENT, + name VARCHAR(30) NOT NULL, + some_update VARCHAR(30), + PRIMARY KEY (id) + )ENGINE=MyISAM; + + CREATE TABLE foobar ( + id INTEGER NOT NULL AUTO_INCREMENT, + user_id INTEGER, + data VARCHAR(30), + some_update VARCHAR(30), + PRIMARY KEY (id), + FOREIGN KEY(user_id) REFERENCES users (id) + )ENGINE=MyISAM;` + mustExec(c, s.testDB, testSQL) + testSQL = ` + INSERT INTO users (id, name, some_update) VALUES + (8, 'ed', 'value'), + (9, 'fred', 'value'); + + INSERT INTO foobar (id, user_id, data) VALUES + (2, 8, 'd1'), + (3, 8, 'd2'), + (4, 9, 'd3');` + mustExec(c, s.testDB, testSQL) + + testSQL = `UPDATE users, foobar SET foobar.data=(concat(foobar.data, 'a')), + foobar.some_update='im the other update', users.name='ed2', users.some_update='im the update' + WHERE users.id = foobar.user_id AND users.name = 'ed';` + r := mustExec(c, s.testDB, testSQL) + checkResult(c, r, 3, 0) +} From 12d5ea7f7d552612c764c8a7f16d4a6c0ed3ea5c Mon Sep 17 00:00:00 2001 From: siddontang Date: Tue, 13 Oct 2015 21:39:08 +0800 Subject: [PATCH 08/58] plans: clone row data to prevent outer modification. --- plan/plans/join.go | 7 ++++++- 1 file changed, 6 insertions(+), 1 deletion(-) 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) From 65c17102c958524b882ae4f470e90e102d7b1c66 Mon Sep 17 00:00:00 2001 From: siddontang Date: Tue, 13 Oct 2015 22:30:33 +0800 Subject: [PATCH 09/58] stmts: split checkUpdateColumns. --- stmt/stmts/insert.go | 16 +++++++++++++++- stmt/stmts/update.go | 35 ++++++++++++++--------------------- 2 files changed, 29 insertions(+), 22 deletions(-) diff --git a/stmt/stmts/insert.go b/stmt/stmts/insert.go index 20f20b93a5..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,7 +233,7 @@ func (s *InsertIntoStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) } insertValueCount := len(s.Lists[0]) - toUpdateColumns, err0 := checkUpdateColumns(s.OnDuplicate, nil, t) + toUpdateColumns, err0 := getOnDuplicateUpdateColumns(s.OnDuplicate, t) if err0 != nil { return nil, errors.Trace(err0) } @@ -322,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/update.go b/stmt/stmts/update.go index dde51aa787..8f11515edf 100644 --- a/stmt/stmts/update.go +++ b/stmt/stmts/update.go @@ -97,31 +97,24 @@ func findColumnByName(t table.Table, name string) (*column.Col, error) { return c, nil } -func checkUpdateColumns(assignList []expression.Assignment, fields []*field.ResultField, t table.Table) (map[int]expression.Assignment, error) { +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 { - if fields != nil { - 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 - } else { - c, err := findColumnByName(t, field.JoinQualifiedName("", v.TableName, v.ColName)) - if err != nil { - return nil, errors.Trace(err) - } - m[c.Offset] = v + 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 m, nil @@ -256,7 +249,7 @@ func (s *UpdateStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) { // Get table alias map. fs := p.GetFields() - columns, err0 := checkUpdateColumns(s.List, fs, nil) + columns, err0 := getUpdateColumns(s.List, fs) if err0 != nil { return nil, errors.Trace(err0) } From 910f283381c55b78da28173fdcd70c6a0e4e7e7f Mon Sep 17 00:00:00 2001 From: Shen Li Date: Tue, 13 Oct 2015 23:05:49 +0800 Subject: [PATCH 10/58] *: Fix bug for info.go and add test case --- plan/plans/info.go | 2 +- tidb-server/server/conn.go | 1 - tidb_test.go | 20 ++++++++++++++++++++ 3 files changed, 21 insertions(+), 2 deletions(-) diff --git a/plan/plans/info.go b/plan/plans/info.go index 3a2b214314..1d75d1943d 100644 --- a/plan/plans/info.go +++ b/plan/plans/info.go @@ -367,7 +367,7 @@ func (isp *InfoSchemaPlan) fetchColumns(schemas []*model.DBInfo) { columnType = fmt.Sprintf("%s(%d)", columnType, colLen) } } else { - columnType = fmt.Sprintf("%s(%d, %d)", columnType, colLen, col.Decimal) + columnType = fmt.Sprintf("%s(%d,%d)", columnType, colLen, col.Decimal) } columnDesc := column.NewColDesc(&column.Col{ColumnInfo: *col}) var columnDefault interface{} diff --git a/tidb-server/server/conn.go b/tidb-server/server/conn.go index 5f19f589fb..26f290b4bd 100644 --- a/tidb-server/server/conn.go +++ b/tidb-server/server/conn.go @@ -433,7 +433,6 @@ func (cc *clientConn) writeResultset(rs ResultSet, binary bool) error { var rowData []byte rowData, err = dumpRowValuesBinary(cc.alloc, columns, row) if err != nil { - fmt.Println("[conn]", err) return errors.Trace(err) } data = append(data, rowData...) diff --git a/tidb_test.go b/tidb_test.go index 71e957420c..84bdc17462 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -1067,6 +1067,26 @@ 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 newSession(c *C, store kv.Storage, dbName string) Session { se, err := CreateSession(store) c.Assert(err, IsNil) From 249fae310ad953590a6cb733760c1399a97e47ef Mon Sep 17 00:00:00 2001 From: siddontang Date: Wed, 14 Oct 2015 10:15:45 +0800 Subject: [PATCH 11/58] stmts: lazy update, cache all select then do update. Fix issue #376 --- stmt/stmts/update.go | 10 +++++-- stmt/stmts/update_test.go | 57 ++++++++++++++++++++++++++++++++++++++- 2 files changed, 64 insertions(+), 3 deletions(-) diff --git a/stmt/stmts/update.go b/stmt/stmts/update.go index 8f11515edf..4ea5da59dc 100644 --- a/stmt/stmts/update.go +++ b/stmt/stmts/update.go @@ -255,6 +255,7 @@ func (s *UpdateStmt) Exec(ctx context.Context) (_ rset.Recordset, err error) { } m := map[interface{}]interface{}{} + var records []*plan.Row for { row, err1 := p.Next(ctx) if err1 != nil { @@ -263,11 +264,16 @@ 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[expression.ExprEvalIdentFunc] = func(name string) (interface{}, error) { return plans.GetIdentValue(name, p.GetFields(), rowData, field.DefaultFieldFlag) diff --git a/stmt/stmts/update_test.go b/stmt/stmts/update_test.go index 34e066202a..422dd687ca 100644 --- a/stmt/stmts/update_test.go +++ b/stmt/stmts/update_test.go @@ -225,7 +225,7 @@ func (s *testStmtSuite) TestIssue345(c *C) { // See https://github.com/pingcap/tidb/issues/369 func (s *testStmtSuite) TestIssue369(c *C) { - testSQL := `drop table if exists users, foobar;` + testSQL := `DROP TABLE IF EXISTS users, foobar;` mustExec(c, s.testDB, testSQL) testSQL = `CREATE TABLE users ( @@ -261,3 +261,58 @@ func (s *testStmtSuite) TestIssue369(c *C) { r := mustExec(c, s.testDB, testSQL) checkResult(c, r, 3, 0) } + +// See https://github.com/pingcap/tidb/issues/376 +func (s *testStmtSuite) TestIssue376(c *C) { + testSQL := ` + DROP TABLE IF EXISTS users, foobar, addresses; + CREATE TABLE users ( + id INTEGER NOT NULL AUTO_INCREMENT, + name VARCHAR(30) NOT NULL, + some_update VARCHAR(30), + PRIMARY KEY (id) + )ENGINE=MyISAM; + + CREATE TABLE foobar ( + id INTEGER NOT NULL AUTO_INCREMENT, + user_id INTEGER, + data VARCHAR(30), + some_update VARCHAR(30), + PRIMARY KEY (id), + FOREIGN KEY(user_id) REFERENCES users (id) + )ENGINE=MyISAM; + + CREATE TABLE addresses ( + id INTEGER NOT NULL AUTO_INCREMENT, + user_id INTEGER, + email_address VARCHAR(50) NOT NULL, + PRIMARY KEY (id), + FOREIGN KEY(user_id) REFERENCES users (id) + )ENGINE=MyISAM; + + INSERT INTO users (id, name, some_update) VALUES + (8, 'ed', 'value'), + (9, 'fred', 'value'); + + INSERT INTO addresses (id, user_id, email_address) VALUES + (2, 8, 'ed@wood.com'), + (3, 8, 'ed@bettyboop.com'), + (4, 9, 'fred@fred.com'); + + INSERT INTO foobar (id, user_id, data) VALUES + (2, 8, 'd1'), + (3, 8, 'd2'), + (4, 9, 'd3');` + + mustExec(c, s.testDB, testSQL) + testSQL = ` + UPDATE addresses, users SET users.name='ed2', users.some_update='im the update', + addresses.email_address=users.name WHERE users.id = addresses.user_id AND users.name = 'ed';` + r := mustExec(c, s.testDB, testSQL) + checkResult(c, r, 3, 0) + + testSQL = `SELECT addresses.id, addresses.user_id, addresses.email_address FROM addresses ORDER BY addresses.id;` + rows, err := s.testDB.Query(testSQL) + c.Assert(err, IsNil) + matchRows(c, rows, [][]interface{}{{2, 8, "ed"}, {3, 8, "ed"}, {4, 9, "fred@fred.com"}}) +} From 3a7e3ed2d036da96710b829d189894984d69fd90 Mon Sep 17 00:00:00 2001 From: shenli Date: Wed, 14 Oct 2015 10:38:18 +0800 Subject: [PATCH 12/58] *: Address comment --- column/column.go | 4 +- field/field_test.go | 6 +-- plan/plans/info.go | 25 +++++------- util/types/convert_test.go | 6 +-- util/types/etc.go | 76 +---------------------------------- util/types/etc_test.go | 64 ++++++++++++++--------------- util/types/field_type.go | 36 ++++++++++------- util/types/field_type_test.go | 14 +++---- 8 files changed, 79 insertions(+), 152 deletions(-) diff --git a/column/column.go b/column/column.go index 10f51297cb..220441d612 100644 --- a/column/column.go +++ b/column/column.go @@ -46,7 +46,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") } @@ -123,7 +123,7 @@ const defaultPrivileges string = "select,insert,update,references" func (c *Col) GetTypeDesc() string { var buf bytes.Buffer - buf.WriteString(types.FieldTypeToStr(c.Tp, c.Charset)) + buf.WriteString(types.TypeToStr(c.Tp, c.Charset)) switch c.Tp { case mysql.TypeSet, mysql.TypeEnum: // Format is ENUM ('e1', 'e2') or SET ('e1', 'e2') 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/plan/plans/info.go b/plan/plans/info.go index 1d75d1943d..d5ce4bccc3 100644 --- a/plan/plans/info.go +++ b/plan/plans/info.go @@ -361,28 +361,21 @@ func (isp *InfoSchemaPlan) fetchColumns(schemas []*model.DBInfo) { if decimal == types.UnspecifiedLength { decimal = 0 } - columnType := types.TypeToStr(col.Tp, col.Charset == charset.CharsetBin) - if col.Decimal == types.UnspecifiedLength { - if colLen != types.UnspecifiedLength { - columnType = fmt.Sprintf("%s(%d)", columnType, colLen) - } - } else { - columnType = fmt.Sprintf("%s(%d,%d)", columnType, colLen, col.Decimal) - } + columnType := col.FieldType.SimpleString() 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/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 b2e7cde4da..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 b1af52f00d..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, false, "text") - testTypeToStr(c, mysql.TypeLongBlob, false, "longtext") - testTypeToStr(c, mysql.TypeTinyBlob, false, "tinytext") - testTypeToStr(c, mysql.TypeMediumBlob, false, "mediumtext") - testTypeToStr(c, mysql.TypeVarchar, true, "varbinary") - testTypeToStr(c, mysql.TypeString, true, "binary") - testTypeToStr(c, mysql.TypeTiny, true, "tinyint") - testTypeToStr(c, mysql.TypeBlob, true, "blob") - testTypeToStr(c, mysql.TypeLongBlob, true, "longblob") - testTypeToStr(c, mysql.TypeTinyBlob, true, "tinyblob") - testTypeToStr(c, mysql.TypeMediumBlob, true, "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..b9a7949f58 100644 --- a/util/types/field_type.go +++ b/util/types/field_type.go @@ -52,45 +52,51 @@ 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} +// SimpleString only consider Tp/CharsetBin/Flen/Deimal. +// This is used for showing column type in infoschema. +func (ft *FieldType) SimpleString() 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, "','"))) + suffix = fmt.Sprintf("('%s')", strings.Join(ft.Elems, "','")) default: if ft.Flen != UnspecifiedLength { if ft.Decimal == UnspecifiedLength { - ans = append(ans, fmt.Sprintf("(%d)", ft.Flen)) + 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.SimpleString()} 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..172a1b6a6e 100644 --- a/util/types/field_type_test.go +++ b/util/types/field_type_test.go @@ -28,33 +28,33 @@ 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.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.TypeSet) ft.Elems = []string{"a", "b"} - c.Assert(ft.String(), Equals, "set ('a','b')") + c.Assert(ft.String(), Equals, "set('a','b')") } From d01821579ed943b87e27d027ee491a5d49016d48 Mon Sep 17 00:00:00 2001 From: shenli Date: Wed, 14 Oct 2015 11:06:09 +0800 Subject: [PATCH 13/58] *: Address comment --- plan/plans/info.go | 2 +- util/types/field_type.go | 10 ++++++---- util/types/field_type_test.go | 13 +++++++++++++ 3 files changed, 20 insertions(+), 5 deletions(-) diff --git a/plan/plans/info.go b/plan/plans/info.go index d5ce4bccc3..1700cd8422 100644 --- a/plan/plans/info.go +++ b/plan/plans/info.go @@ -361,7 +361,7 @@ func (isp *InfoSchemaPlan) fetchColumns(schemas []*model.DBInfo) { if decimal == types.UnspecifiedLength { decimal = 0 } - columnType := col.FieldType.SimpleString() + columnType := col.FieldType.TypeString() columnDesc := column.NewColDesc(&column.Col{ColumnInfo: *col}) var columnDefault interface{} if columnDesc.DefaultValue != nil { diff --git a/util/types/field_type.go b/util/types/field_type.go index b9a7949f58..7eab28ae47 100644 --- a/util/types/field_type.go +++ b/util/types/field_type.go @@ -52,9 +52,9 @@ func NewFieldType(tp byte) *FieldType { } } -// SimpleString only consider Tp/CharsetBin/Flen/Deimal. +// TypeString only consider Tp/CharsetBin/Flen/Deimal. // This is used for showing column type in infoschema. -func (ft *FieldType) SimpleString() string { +func (ft *FieldType) TypeString() string { ts := TypeToStr(ft.Tp, ft.Charset) suffix := "" switch ft.Tp { @@ -64,7 +64,9 @@ func (ft *FieldType) SimpleString() string { default: if ft.Flen != UnspecifiedLength { if ft.Decimal == UnspecifiedLength { - suffix = fmt.Sprintf("(%d)", ft.Flen) + if ft.Tp != mysql.TypeFloat && ft.Tp != mysql.TypeDouble { + suffix = fmt.Sprintf("(%d)", ft.Flen) + } } else { suffix = fmt.Sprintf("(%d,%d)", ft.Flen, ft.Decimal) } @@ -78,7 +80,7 @@ func (ft *FieldType) SimpleString() string { // String joins the information of FieldType and // returns a string. func (ft *FieldType) String() string { - strs := []string{ft.SimpleString()} + strs := []string{ft.TypeString()} if mysql.HasUnsignedFlag(ft.Flag) { strs = append(strs, "UNSIGNED") } diff --git a/util/types/field_type_test.go b/util/types/field_type_test.go index 172a1b6a6e..c00120d71d 100644 --- a/util/types/field_type_test.go +++ b/util/types/field_type_test.go @@ -38,6 +38,19 @@ func (s *testFieldTypeSuite) TestFieldType(c *C) { ft.Flen = 10 ft.Decimal = 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 From c1f168673f4da1f32d540736e82521865c696650 Mon Sep 17 00:00:00 2001 From: shenli Date: Wed, 14 Oct 2015 11:30:36 +0800 Subject: [PATCH 14/58] *: Address comment --- plan/plans/info.go | 2 +- util/types/field_type.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/plan/plans/info.go b/plan/plans/info.go index 1700cd8422..9b222c7a6a 100644 --- a/plan/plans/info.go +++ b/plan/plans/info.go @@ -361,7 +361,7 @@ func (isp *InfoSchemaPlan) fetchColumns(schemas []*model.DBInfo) { if decimal == types.UnspecifiedLength { decimal = 0 } - columnType := col.FieldType.TypeString() + columnType := col.FieldType.CompactStr() columnDesc := column.NewColDesc(&column.Col{ColumnInfo: *col}) var columnDefault interface{} if columnDesc.DefaultValue != nil { diff --git a/util/types/field_type.go b/util/types/field_type.go index 7eab28ae47..b5fbb1f19a 100644 --- a/util/types/field_type.go +++ b/util/types/field_type.go @@ -52,9 +52,9 @@ func NewFieldType(tp byte) *FieldType { } } -// TypeString only consider Tp/CharsetBin/Flen/Deimal. +// CompactStr only consider Tp/CharsetBin/Flen/Deimal. // This is used for showing column type in infoschema. -func (ft *FieldType) TypeString() string { +func (ft *FieldType) CompactStr() string { ts := TypeToStr(ft.Tp, ft.Charset) suffix := "" switch ft.Tp { @@ -80,7 +80,7 @@ func (ft *FieldType) TypeString() string { // String joins the information of FieldType and // returns a string. func (ft *FieldType) String() string { - strs := []string{ft.TypeString()} + strs := []string{ft.CompactStr()} if mysql.HasUnsignedFlag(ft.Flag) { strs = append(strs, "UNSIGNED") } From a13e24759e3cd290919d208cfdce90ddcc65425d Mon Sep 17 00:00:00 2001 From: siddontang Date: Wed, 14 Oct 2015 11:55:10 +0800 Subject: [PATCH 15/58] stmts: simplify test. --- stmt/stmts/update_test.go | 107 +++++++------------------------------- 1 file changed, 19 insertions(+), 88 deletions(-) diff --git a/stmt/stmts/update_test.go b/stmt/stmts/update_test.go index 422dd687ca..8582faf65f 100644 --- a/stmt/stmts/update_test.go +++ b/stmt/stmts/update_test.go @@ -223,96 +223,27 @@ func (s *testStmtSuite) TestIssue345(c *C) { mustCommit(c, tx) } -// See https://github.com/pingcap/tidb/issues/369 -func (s *testStmtSuite) TestIssue369(c *C) { - testSQL := `DROP TABLE IF EXISTS users, foobar;` - mustExec(c, s.testDB, testSQL) - - testSQL = `CREATE TABLE users ( - id INTEGER NOT NULL AUTO_INCREMENT, - name VARCHAR(30) NOT NULL, - some_update VARCHAR(30), - PRIMARY KEY (id) - )ENGINE=MyISAM; - - CREATE TABLE foobar ( - id INTEGER NOT NULL AUTO_INCREMENT, - user_id INTEGER, - data VARCHAR(30), - some_update VARCHAR(30), - PRIMARY KEY (id), - FOREIGN KEY(user_id) REFERENCES users (id) - )ENGINE=MyISAM;` - mustExec(c, s.testDB, testSQL) - testSQL = ` - INSERT INTO users (id, name, some_update) VALUES - (8, 'ed', 'value'), - (9, 'fred', 'value'); - - INSERT INTO foobar (id, user_id, data) VALUES - (2, 8, 'd1'), - (3, 8, 'd2'), - (4, 9, 'd3');` - mustExec(c, s.testDB, testSQL) - - testSQL = `UPDATE users, foobar SET foobar.data=(concat(foobar.data, 'a')), - foobar.some_update='im the other update', users.name='ed2', users.some_update='im the update' - WHERE users.id = foobar.user_id AND users.name = 'ed';` - r := mustExec(c, s.testDB, testSQL) - checkResult(c, r, 3, 0) -} - -// See https://github.com/pingcap/tidb/issues/376 -func (s *testStmtSuite) TestIssue376(c *C) { +func (s *testStmtSuite) TestMultiUpdate(c *C) { + // fix https://github.com/pingcap/tidb/issues/369 testSQL := ` - DROP TABLE IF EXISTS users, foobar, addresses; - CREATE TABLE users ( - id INTEGER NOT NULL AUTO_INCREMENT, - name VARCHAR(30) NOT NULL, - some_update VARCHAR(30), - PRIMARY KEY (id) - )ENGINE=MyISAM; - - CREATE TABLE foobar ( - id INTEGER NOT NULL AUTO_INCREMENT, - user_id INTEGER, - data VARCHAR(30), - some_update VARCHAR(30), - PRIMARY KEY (id), - FOREIGN KEY(user_id) REFERENCES users (id) - )ENGINE=MyISAM; - - CREATE TABLE addresses ( - id INTEGER NOT NULL AUTO_INCREMENT, - user_id INTEGER, - email_address VARCHAR(50) NOT NULL, - PRIMARY KEY (id), - FOREIGN KEY(user_id) REFERENCES users (id) - )ENGINE=MyISAM; - - INSERT INTO users (id, name, some_update) VALUES - (8, 'ed', 'value'), - (9, 'fred', 'value'); - - INSERT INTO addresses (id, user_id, email_address) VALUES - (2, 8, 'ed@wood.com'), - (3, 8, 'ed@bettyboop.com'), - (4, 9, 'fred@fred.com'); - - INSERT INTO foobar (id, user_id, data) VALUES - (2, 8, 'd1'), - (3, 8, 'd2'), - (4, 9, 'd3');` - + 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) - testSQL = ` - UPDATE addresses, users SET users.name='ed2', users.some_update='im the update', - addresses.email_address=users.name WHERE users.id = addresses.user_id AND users.name = 'ed';` - r := mustExec(c, s.testDB, testSQL) - checkResult(c, r, 3, 0) - testSQL = `SELECT addresses.id, addresses.user_id, addresses.email_address FROM addresses ORDER BY addresses.id;` - rows, err := s.testDB.Query(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{}{{2, 8, "ed"}, {3, 8, "ed"}, {4, 9, "fred@fred.com"}}) + matchRows(c, rows, [][]interface{}{{10}, {10}}) } From 25b571bf591253d5740381fd076892a9e9f54cb9 Mon Sep 17 00:00:00 2001 From: shenli Date: Wed, 14 Oct 2015 11:58:48 +0800 Subject: [PATCH 16/58] *: Set/Enum element with single-quote should be replaced to two single-quotes Address comment and add testcase --- util/types/field_type.go | 7 ++++++- util/types/field_type_test.go | 8 ++++++++ 2 files changed, 14 insertions(+), 1 deletion(-) diff --git a/util/types/field_type.go b/util/types/field_type.go index b5fbb1f19a..30d8efda10 100644 --- a/util/types/field_type.go +++ b/util/types/field_type.go @@ -60,7 +60,12 @@ func (ft *FieldType) CompactStr() string { switch ft.Tp { case mysql.TypeEnum, mysql.TypeSet: // Format is ENUM ('e1', 'e2') or SET ('e1', 'e2') - suffix = 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, "','")) default: if ft.Flen != UnspecifiedLength { if ft.Decimal == UnspecifiedLength { diff --git a/util/types/field_type_test.go b/util/types/field_type_test.go index c00120d71d..ee9c183d8a 100644 --- a/util/types/field_type_test.go +++ b/util/types/field_type_test.go @@ -67,7 +67,15 @@ func (s *testFieldTypeSuite) TestFieldType(c *C) { ft.Elems = []string{"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')") + + ft = NewFieldType(mysql.TypeSet) + ft.Elems = []string{"'a'", "'b'"} + c.Assert(ft.String(), Equals, "set('''a''','''b''')") } From 641aeabd421e259416b1618b612d261c24be6308 Mon Sep 17 00:00:00 2001 From: shenli Date: Wed, 14 Oct 2015 13:23:12 +0800 Subject: [PATCH 17/58] *: Address comment --- tidb-server/server/driver_tidb.go | 5 +++-- util/types/field_type.go | 2 +- 2 files changed, 4 insertions(+), 3 deletions(-) diff --git a/tidb-server/server/driver_tidb.go b/tidb-server/server/driver_tidb.go index f536ca92ec..3fda576361 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. @@ -272,12 +273,12 @@ func convertColumnInfo(fld *field.ResultField) (ci *ColumnInfo) { ci.Schema = fld.DBName ci.Flag = uint16(fld.Flag) ci.Charset = uint16(mysql.CharsetIDs[fld.Charset]) - if fld.Flen == -1 { + if fld.Flen == types.UnspecifiedLength { ci.ColumnLength = 0 } else { ci.ColumnLength = uint32(fld.Flen) } - if fld.Decimal == -1 { + if fld.Decimal == types.UnspecifiedLength { ci.Decimal = 0 } else { ci.Decimal = uint8(fld.Decimal) diff --git a/util/types/field_type.go b/util/types/field_type.go index 30d8efda10..681aacbac9 100644 --- a/util/types/field_type.go +++ b/util/types/field_type.go @@ -52,7 +52,7 @@ func NewFieldType(tp byte) *FieldType { } } -// CompactStr only consider Tp/CharsetBin/Flen/Deimal. +// 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) From 9acfcd2b1454d1c65c618690e42c914de9ceedb6 Mon Sep 17 00:00:00 2001 From: shenli Date: Wed, 14 Oct 2015 13:49:50 +0800 Subject: [PATCH 18/58] *: Address comment --- column/column.go | 38 +++-------------------------------- util/types/field_type.go | 4 ++++ util/types/field_type_test.go | 26 ++++++++++++++++++++++++ 3 files changed, 33 insertions(+), 35 deletions(-) diff --git a/column/column.go b/column/column.go index 220441d612..6acdfe2c37 100644 --- a/column/column.go +++ b/column/column.go @@ -18,8 +18,6 @@ package column import ( - "bytes" - "fmt" "strings" "github.com/juju/errors" @@ -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.TypeToStr(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/util/types/field_type.go b/util/types/field_type.go index 681aacbac9..ec96912bcf 100644 --- a/util/types/field_type.go +++ b/util/types/field_type.go @@ -66,6 +66,10 @@ func (ft *FieldType) CompactStr() string { 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 { diff --git a/util/types/field_type_test.go b/util/types/field_type_test.go index ee9c183d8a..558c799d8a 100644 --- a/util/types/field_type_test.go +++ b/util/types/field_type_test.go @@ -78,4 +78,30 @@ func (s *testFieldTypeSuite) TestFieldType(c *C) { 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") } From 284219a641f3e767a60e3ac576e7128b5d49a8a2 Mon Sep 17 00:00:00 2001 From: siddontang Date: Wed, 14 Oct 2015 14:30:56 +0800 Subject: [PATCH 19/58] plans: check error for show plan. --- plan/plans/info_test.go | 3 +++ plan/plans/show.go | 4 +++- plan/plans/show_test.go | 5 +++++ 3 files changed, 11 insertions(+), 1 deletion(-) 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/show.go b/plan/plans/show.go index a0d1e04464..0dac38c217 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 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) } From ed13467c7f8ec2402a6b412d6f79cc5ffd014bee Mon Sep 17 00:00:00 2001 From: siddontang Date: Wed, 14 Oct 2015 16:26:22 +0800 Subject: [PATCH 20/58] *: rename mysql err to common format --- mysqldef/errcode.go | 1732 ++++++++++++++--------------- mysqldef/errname.go | 1728 ++++++++++++++-------------- mysqldef/error_test.go | 4 +- mysqldef/state.go | 450 ++++---- tidb-server/server/conn.go | 11 +- tidb-server/server/conn_stmt.go | 8 +- tidb-server/server/driver_tidb.go | 2 +- 7 files changed, 1968 insertions(+), 1967 deletions(-) diff --git a/mysqldef/errcode.go b/mysqldef/errcode.go index 4f40c4f66f..839a8ac3b7 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 + ErrConCountError = 1040 + ErrOutOfResources = 1041 + ErrBadHostError = 1042 + ErrHandshakeError = 1043 + ErrDbaccessDeniedError = 1044 + ErrAccessDeniedError = 1045 + ErrNoDbError = 1046 + ErrUnknownComError = 1047 + ErrBadNullError = 1048 + ErrBadDbError = 1049 + ErrTableExistsError = 1050 + ErrBadTableError = 1051 + ErrNonUniqError = 1052 + ErrServerShutdown = 1053 + ErrBadFieldError = 1054 + ErrWrongFieldWithGroup = 1055 + ErrWrongGroupField = 1056 + ErrWrongSumSelect = 1057 + ErrWrongValueCount = 1058 + ErrTooLongIdent = 1059 + ErrDupFieldname = 1060 + ErrDupKeyname = 1061 + ErrDupEntry = 1062 + ErrWrongFieldSpec = 1063 + ErrParseError = 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 + ErrIpsockError = 1081 + ErrNoSuchIndex = 1082 + ErrWrongFieldTerminators = 1083 + ErrBlobsAndNoTerminated = 1084 + ErrTextfileNotReadable = 1085 + ErrFileExistsError = 1086 + ErrLoadInfo = 1087 + ErrAlterInfo = 1088 + ErrWrongSubKey = 1089 + ErrCantRemoveAllFields = 1090 + ErrCantDropFieldOrKey = 1091 + ErrInsertInfo = 1092 + ErrUpdateTableUsed = 1093 + ErrNoSuchThread = 1094 + ErrKillDeniedError = 1095 + ErrNoTablesUsed = 1096 + ErrTooBigSet = 1097 + ErrNoUniqueLogfile = 1098 + ErrTableNotLockedForWrite = 1099 + ErrTableNotLocked = 1100 + ErrBlobCantHaveDefault = 1101 + ErrWrongDbName = 1102 + ErrWrongTableName = 1103 + ErrTooBigSelect = 1104 + ErrUnknownError = 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 + ErrRegexpError = 1139 + ErrMixOfGroupFuncAndFields = 1140 + ErrNonexistingGrant = 1141 + ErrTableaccessDeniedError = 1142 + ErrColumnaccessDeniedError = 1143 + ErrIllegalGrantForTable = 1144 + ErrGrantWrongHostOrUser = 1145 + ErrNoSuchTable = 1146 + ErrNonexistingTableGrant = 1147 + ErrNotAllowedCommand = 1148 + ErrSyntaxError = 1149 + ErrDelayedCantChangeLock = 1150 + ErrTooManyDelayedThreads = 1151 + ErrAbortingConnection = 1152 + ErrNetPacketTooLarge = 1153 + ErrNetReadErrorFromPipe = 1154 + ErrNetFcntlError = 1155 + ErrNetPacketsOutOfOrder = 1156 + ErrNetUncompressError = 1157 + ErrNetReadError = 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 + ErrSpecificAccessDeniedError = 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 + ErrZlibZMemError = 1257 + ErrZlibZBufError = 1258 + ErrZlibZDataError = 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 + ErrProcaccessDeniedError = 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 + ErrPartitionRequiresValuesError = 1479 + ErrPartitionWrongValuesError = 1480 + ErrPartitionMaxvalueError = 1481 + ErrPartitionSubpartitionError = 1482 + ErrPartitionSubpartMixError = 1483 + ErrPartitionWrongNoPartError = 1484 + ErrPartitionWrongNoSubpartError = 1485 + ErrWrongExprInPartitionFuncError = 1486 + ErrNoConstExprInRangeOrListError = 1487 + ErrFieldNotFoundPartError = 1488 + ErrListOfFieldsOnlyInHashError = 1489 + ErrInconsistentPartitionInfoError = 1490 + ErrPartitionFuncNotAllowedError = 1491 + ErrPartitionsMustBeDefinedError = 1492 + ErrRangeNotIncreasingError = 1493 + ErrInconsistentTypeOfFunctionsError = 1494 + ErrMultipleDefConstInListPartError = 1495 + ErrPartitionEntryError = 1496 + ErrMixHandlerError = 1497 + ErrPartitionNotDefinedError = 1498 + ErrTooManyPartitionsError = 1499 + ErrSubpartitionError = 1500 + ErrCantCreateHandlerFile = 1501 + ErrBlobFieldInPartFuncError = 1502 + ErrUniqueKeyNeedAllFieldsInPf = 1503 + ErrNoPartsError = 1504 + ErrPartitionMgmtOnNonpartitioned = 1505 + ErrForeignKeyOnPartitioned = 1506 + ErrDropPartitionNonExistent = 1507 + ErrDropLastPartition = 1508 + ErrCoalesceOnlyOnHashPartition = 1509 + ErrReorgHashOnlyOnSameNo = 1510 + ErrReorgNoParamError = 1511 + ErrOnlyOnRangeListPartition = 1512 + ErrAddPartitionSubpartError = 1513 + ErrAddPartitionNoNewPartition = 1514 + ErrCoalescePartitionNoPartition = 1515 + ErrReorgPartitionNotExist = 1516 + ErrSameNamePartition = 1517 + ErrNoBinlogError = 1518 + ErrConsecutiveReorgPartitions = 1519 + ErrReorgOutsideRange = 1520 + ErrPartitionFunctionFailure = 1521 + ErrPartStateError = 1522 + ErrLimitedPartRange = 1523 + ErrPluginIsNotLoaded = 1524 + ErrWrongValue = 1525 + ErrNoPartitionForGivenValue = 1526 + ErrFilegroupOptionOnlyOnce = 1527 + ErrCreateFilegroupFailed = 1528 + ErrDropFilegroupFailed = 1529 + ErrTablespaceAutoExtendError = 1530 + ErrWrongSizeNumber = 1531 + ErrSizeOverflowError = 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 + ErrEventCompileError = 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 + ErrPartitionConstDomainError = 1563 + ErrPartitionFunctionIsNotAllowed = 1564 + ErrDdlLogError = 1565 + ErrNullInValuesLessThan = 1566 + ErrWrongPartitionName = 1567 + ErrCantChangeTxCharacteristics = 1568 + ErrDupEntryAutoincrementCase = 1569 + ErrEventModifyQueueError = 1570 + ErrEventSetVarError = 1571 + ErrPartitionMergeError = 1572 + ErrCantActivateLog = 1573 + ErrRbrNotAvailable = 1574 + ErrBase64DecodeError = 1575 + ErrEventRecursionForbidden = 1576 + ErrEventsDbError = 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 + ErrSlaveFatalError = 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 + ErrNdbReplicationSchemaError = 1625 + ErrConflictFnParseError = 1626 + ErrExceptionsWriteError = 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 + ErrPartitionColumnListError = 1653 + ErrWrongTypeColumnValueError = 1654 + ErrTooManyPartitionFuncFieldsError = 1655 + ErrMaxvalueInValuesIn = 1656 + ErrTooManyValuesError = 1657 + ErrRowSinglePartitionFieldError = 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 + ErrValuesIsNotIntTypeError = 1697 + ErrAccessDeniedNoPasswordError = 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 + ErrIoReadError = 1810 + ErrIoWriteError = 1811 + ErrTablespaceMissing = 1812 + ErrTablespaceExists = 1813 + ErrTablespaceDiscarded = 1814 + ErrInternalError = 1815 + ErrInnodbImportError = 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..7ecc731246 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)", + ErrConCountError: "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", + ErrBadHostError: "Can't get hostname for your address", + ErrHandshakeError: "Bad handshake", + ErrDbaccessDeniedError: "Access denied for user '%-.48s'@'%-.64s' to database '%-.192s'", + ErrAccessDeniedError: "Access denied for user '%-.48s'@'%-.64s' (using password: %s)", + ErrNoDbError: "No database selected", + ErrUnknownComError: "Unknown command", + ErrBadNullError: "Column '%-.192s' cannot be null", + ErrBadDbError: "Unknown database '%-.192s'", + ErrTableExistsError: "Table '%-.192s' already exists", + ErrBadTableError: "Unknown table '%-.100s'", + ErrNonUniqError: "Column '%-.192s' in %-.192s is ambiguous", + ErrServerShutdown: "Server shutdown in progress", + ErrBadFieldError: "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'", + ErrParseError: "%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", + ErrIpsockError: "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", + ErrFileExistsError: "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", + ErrKillDeniedError: "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", + ErrUnknownError: "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", + ErrRegexpError: "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'", + ErrTableaccessDeniedError: "%-.128s command denied to user '%-.48s'@'%-.64s' for table '%-.64s'", + ErrColumnaccessDeniedError: "%-.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", + ErrSyntaxError: "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", + ErrNetFcntlError: "Got an error from fcntl()", + ErrNetPacketsOutOfOrder: "Got packets out of order", + ErrNetUncompressError: "Couldn't uncompress communication packet", + ErrNetReadError: "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)", + ErrSpecificAccessDeniedError: "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)", + ErrZlibZMemError: "ZLIB: Not enough memory", + ErrZlibZBufError: "ZLIB: Not enough room in the output buffer (probably, length of uncompressed data was corrupted)", + ErrZlibZDataError: "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'", + ErrProcaccessDeniedError: "%-.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 error: %-.64s", + ErrForeignDataSourceDoesntExist: "The foreign data source you are trying to reference does not exist. Data source error: %-.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 error: %-.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 error: %-.64s", + ErrIllegalHaCreateOption: "Table storage engine '%-.64s' does not support the create option '%.64s'", + ErrPartitionRequiresValuesError: "Syntax error: %-.64s PARTITIONING requires definition of VALUES %-.64s for each partition", + ErrPartitionWrongValuesError: "Only %-.64s PARTITIONING can use VALUES %-.64s in partition definition", + ErrPartitionMaxvalueError: "MAXVALUE can only be used in last partition definition", + ErrPartitionSubpartitionError: "Subpartitions can only be hash partitions and by key", + ErrPartitionSubpartMixError: "Must define subpartitions on all partitions if on one partition", + ErrPartitionWrongNoPartError: "Wrong number of partitions defined, mismatch with previous setting", + ErrPartitionWrongNoSubpartError: "Wrong number of subpartitions defined, mismatch with previous setting", + ErrWrongExprInPartitionFuncError: "Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed", + ErrNoConstExprInRangeOrListError: "Expression in RANGE/LIST VALUES must be constant", + ErrFieldNotFoundPartError: "Field in list of fields for partition function not found in table", + ErrListOfFieldsOnlyInHashError: "List of fields is only allowed in KEY partitions", + ErrInconsistentPartitionInfoError: "The partition info in the frm file is not consistent with what can be written into the frm file", + ErrPartitionFuncNotAllowedError: "The %-.192s function returns the wrong type", + ErrPartitionsMustBeDefinedError: "For %-.64s partitions each partition must be defined", + ErrRangeNotIncreasingError: "VALUES LESS THAN value must be strictly increasing for each partition", + ErrInconsistentTypeOfFunctionsError: "VALUES value must be of same type as partition function", + ErrMultipleDefConstInListPartError: "Multiple definition of same constant in list partitioning", + ErrPartitionEntryError: "Partitioning can not be used stand-alone in query", + ErrMixHandlerError: "The mix of handlers in the partitions is not allowed in this version of MySQL", + ErrPartitionNotDefinedError: "For the partitioned engine it is necessary to define all %-.64s", + ErrTooManyPartitionsError: "Too many partitions (including subpartitions) were defined", + ErrSubpartitionError: "It is only possible to mix RANGE/LIST partitioning with HASH/KEY partitioning for subpartitioning", + ErrCantCreateHandlerFile: "Failed to create specific handler file", + ErrBlobFieldInPartFuncError: "A BLOB field is not allowed in partition function", + ErrUniqueKeyNeedAllFieldsInPf: "A %-.192s must include all columns in the table's partitioning function", + ErrNoPartsError: "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", + ErrReorgNoParamError: "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", + ErrAddPartitionSubpartError: "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", + ErrNoBinlogError: "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", + ErrPartStateError: "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", + ErrTablespaceAutoExtendError: "The handler doesn't support autoextend of tablespaces", + ErrWrongSizeNumber: "A size parameter was incorrectly specified, either number or on the form 10M", + ErrSizeOverflowError: "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", + ErrEventCompileError: "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", + ErrPartitionConstDomainError: "Partition constant is out of partition function domain", + ErrPartitionFunctionIsNotAllowed: "This partition function is not allowed", + ErrDdlLogError: "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'", + ErrEventModifyQueueError: "Internal scheduler error %d", + ErrEventSetVarError: "Error during starting/stopping of the scheduler. Error code %u", + ErrPartitionMergeError: "Engine cannot be used in partitioned tables", + ErrCantActivateLog: "Cannot activate '%-.64s' log", + ErrRbrNotAvailable: "The server was not built with row-based replication", + ErrBase64DecodeError: "Decoding of base64 string failed", + ErrEventRecursionForbidden: "Recursion of EVENT DDL statements is forbidden when body is present", + ErrEventsDbError: "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", + ErrSlaveFatalError: "Fatal error: %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).", + ErrNdbReplicationSchemaError: "Bad schema for mysql.ndbReplication table. Message: %-.64s", + ErrConflictFnParseError: "Error in parsing conflict function. Message: %-.64s", + ErrExceptionsWriteError: "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'", + ErrPartitionColumnListError: "Inconsistency in usage of column lists for partitioning", + ErrWrongTypeColumnValueError: "Partition column values of incorrect type", + ErrTooManyPartitionFuncFieldsError: "Too many fields in '%-.192s'", + ErrMaxvalueInValuesIn: "Cannot use MAXVALUE as value in VALUES IN", + ErrTooManyValuesError: "Cannot have more than one value for this type of %-.64s partitioning", + ErrRowSinglePartitionFieldError: "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", + ErrValuesIsNotIntTypeError: "VALUES value for partition '%-.64s' must have type INT", + ErrAccessDeniedNoPasswordError: "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", + ErrIoReadError: "IO Read error: (%lu, %s) %s", + ErrIoWriteError: "IO Write error: (%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'", + ErrInternalError: "Internal error: %s", + ErrInnodbImportError: "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_test.go b/mysqldef/error_test.go index eb8a054a6c..d08c104ed3 100644 --- a/mysqldef/error_test.go +++ b/mysqldef/error_test.go @@ -23,13 +23,13 @@ type testSQLErrorSuite struct { } func (s *testSQLErrorSuite) TestSQLError(c *C) { - e := NewError(ErNoDbError, "no db error") + e := NewError(ErrNoDbError, "no db error") c.Assert(len(e.Error()), Greater, 0) e = NewError(0, "customized error") c.Assert(len(e.Error()), Greater, 0) - e = NewDefaultError(ErNoDbError) + e = NewDefaultError(ErrNoDbError) c.Assert(len(e.Error()), Greater, 0) e = NewDefaultError(0, "customized error") diff --git a/mysqldef/state.go b/mysqldef/state.go index 82c3694d23..7d581c1ac4 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", + ErrConCountError: "08004", + ErrBadHostError: "08S01", + ErrHandshakeError: "08S01", + ErrDbaccessDeniedError: "42000", + ErrAccessDeniedError: "28000", + ErrNoDbError: "3D000", + ErrUnknownComError: "08S01", + ErrBadNullError: "23000", + ErrBadDbError: "42000", + ErrTableExistsError: "42S01", + ErrBadTableError: "42S02", + ErrNonUniqError: "23000", + ErrServerShutdown: "08S01", + ErrBadFieldError: "42S22", + ErrWrongFieldWithGroup: "42000", + ErrWrongSumSelect: "42000", + ErrWrongGroupField: "42000", + ErrWrongValueCount: "21S01", + ErrTooLongIdent: "42000", + ErrDupFieldname: "42S21", + ErrDupKeyname: "42000", + ErrDupEntry: "23000", + ErrWrongFieldSpec: "42000", + ErrParseError: "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", + ErrIpsockError: "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", + ErrRegexpError: "42000", + ErrMixOfGroupFuncAndFields: "42000", + ErrNonexistingGrant: "42000", + ErrTableaccessDeniedError: "42000", + ErrColumnaccessDeniedError: "42000", + ErrIllegalGrantForTable: "42000", + ErrGrantWrongHostOrUser: "42000", + ErrNoSuchTable: "42S02", + ErrNonexistingTableGrant: "42000", + ErrNotAllowedCommand: "42000", + ErrSyntaxError: "42000", + ErrAbortingConnection: "08S01", + ErrNetPacketTooLarge: "08S01", + ErrNetReadErrorFromPipe: "08S01", + ErrNetFcntlError: "08S01", + ErrNetPacketsOutOfOrder: "08S01", + ErrNetUncompressError: "08S01", + ErrNetReadError: "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", + ErrSpecificAccessDeniedError: "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", + ErrProcaccessDeniedError: "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", + ErrAccessDeniedNoPasswordError: "28000", + ErrTruncateIllegalFk: "42000", + ErrDaInvalidConditionNumber: "35000", + ErrForeignDuplicateKeyWithChildInfo: "23000", + ErrForeignDuplicateKeyWithoutChildInfo: "23000", + ErrCantExecuteInReadOnlyTransaction: "25006", + ErrAlterOperationNotSupported: "0A000", + ErrAlterOperationNotSupportedReason: "0A000", + ErrDupUnknownInIndex: "23000", } diff --git a/tidb-server/server/conn.go b/tidb-server/server/conn.go index c1056d917c..1083137969 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.NewDefaultError(mysql.ErrAccessDeniedError, 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.NewDefaultError(mysql.ErrAccessDeniedError, cc.user, host, "Yes")) } } return nil @@ -284,7 +284,7 @@ func (cc *clientConn) dispatch(data []byte) error { return cc.handleStmtReset(data) default: msg := fmt.Sprintf("command %d not supported now", cmd) - return mysql.NewError(mysql.ErUnknownError, msg) + return mysql.NewError(mysql.ErrUnknownError, msg) } } @@ -322,8 +322,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.NewError(mysql.ErrUnknownError, 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..a40ee4a1fe 100644 --- a/tidb-server/server/conn_stmt.go +++ b/tidb-server/server/conn_stmt.go @@ -113,7 +113,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.NewDefaultError(mysql.ErrUnknownStmtHandler, strconv.FormatUint(uint64(stmtID), 10), "stmt_execute") } @@ -121,7 +121,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.NewError(mysql.ErrUnknownError, fmt.Sprintf("unsupported flag %d", flag)) } //skip iteration-count, always 1 @@ -324,7 +324,7 @@ func (cc *clientConn) handleStmtSendLongData(data []byte) (err error) { stmt := cc.ctx.GetStatement(stmtID) if stmt == nil { - return mysql.NewDefaultError(mysql.ErUnknownStmtHandler, + return mysql.NewDefaultError(mysql.ErrUnknownStmtHandler, strconv.Itoa(stmtID), "stmt_send_longdata") } @@ -340,7 +340,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.NewDefaultError(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..adf01d7a22 100644 --- a/tidb-server/server/driver_tidb.go +++ b/tidb-server/server/driver_tidb.go @@ -78,7 +78,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.NewDefaultError(mysql.ErrWrongArguments, "stmt_send_longdata") } ts.boundParams[paramID] = append(ts.boundParams[paramID], data...) return nil From ac3650e16bf76bc4179162ff0e4037313b99a119 Mon Sep 17 00:00:00 2001 From: siddontang Date: Wed, 14 Oct 2015 17:08:15 +0800 Subject: [PATCH 21/58] *: Address comment --- mysqldef/errcode.go | 166 +++++++++++++++--------------- mysqldef/errname.go | 174 ++++++++++++++++---------------- mysqldef/error_test.go | 4 +- mysqldef/state.go | 50 ++++----- tidb-server/server/conn.go | 8 +- tidb-server/server/conn_stmt.go | 2 +- 6 files changed, 202 insertions(+), 202 deletions(-) diff --git a/mysqldef/errcode.go b/mysqldef/errcode.go index 839a8ac3b7..d486c36c90 100644 --- a/mysqldef/errcode.go +++ b/mysqldef/errcode.go @@ -57,21 +57,21 @@ const ( ErrOutofmemory = 1037 ErrOutOfSortmemory = 1038 ErrUnexpectedEOF = 1039 - ErrConCountError = 1040 + ErrConCount = 1040 ErrOutOfResources = 1041 - ErrBadHostError = 1042 - ErrHandshakeError = 1043 - ErrDbaccessDeniedError = 1044 - ErrAccessDeniedError = 1045 - ErrNoDbError = 1046 - ErrUnknownComError = 1047 - ErrBadNullError = 1048 - ErrBadDbError = 1049 - ErrTableExistsError = 1050 - ErrBadTableError = 1051 - ErrNonUniqError = 1052 + ErrBadHost = 1042 + ErrHandshake = 1043 + ErrDbaccessDenied = 1044 + ErrAccessDenied = 1045 + ErrNoDb = 1046 + ErrUnknownCom = 1047 + ErrBadNull = 1048 + ErrBadDb = 1049 + ErrTableExists = 1050 + ErrBadTable = 1051 + ErrNonUniq = 1052 ErrServerShutdown = 1053 - ErrBadFieldError = 1054 + ErrBadField = 1054 ErrWrongFieldWithGroup = 1055 ErrWrongGroupField = 1056 ErrWrongSumSelect = 1057 @@ -81,7 +81,7 @@ const ( ErrDupKeyname = 1061 ErrDupEntry = 1062 ErrWrongFieldSpec = 1063 - ErrParseError = 1064 + ErrParse = 1064 ErrEmptyQuery = 1065 ErrNonuniqTable = 1066 ErrInvalidDefault = 1067 @@ -98,12 +98,12 @@ const ( ErrGotSignal = 1078 ErrShutdownComplete = 1079 ErrForcingClose = 1080 - ErrIpsockError = 1081 + ErrIpsock = 1081 ErrNoSuchIndex = 1082 ErrWrongFieldTerminators = 1083 ErrBlobsAndNoTerminated = 1084 ErrTextfileNotReadable = 1085 - ErrFileExistsError = 1086 + ErrFileExists = 1086 ErrLoadInfo = 1087 ErrAlterInfo = 1088 ErrWrongSubKey = 1089 @@ -112,7 +112,7 @@ const ( ErrInsertInfo = 1092 ErrUpdateTableUsed = 1093 ErrNoSuchThread = 1094 - ErrKillDeniedError = 1095 + ErrKillDenied = 1095 ErrNoTablesUsed = 1096 ErrTooBigSet = 1097 ErrNoUniqueLogfile = 1098 @@ -122,7 +122,7 @@ const ( ErrWrongDbName = 1102 ErrWrongTableName = 1103 ErrTooBigSelect = 1104 - ErrUnknownError = 1105 + ErrUnknown = 1105 ErrUnknownProcedure = 1106 ErrWrongParamcountToProcedure = 1107 ErrWrongParametersToProcedure = 1108 @@ -156,26 +156,26 @@ const ( ErrWrongValueCountOnRow = 1136 ErrCantReopenTable = 1137 ErrInvalidUseOfNull = 1138 - ErrRegexpError = 1139 + ErrRegexp = 1139 ErrMixOfGroupFuncAndFields = 1140 ErrNonexistingGrant = 1141 - ErrTableaccessDeniedError = 1142 - ErrColumnaccessDeniedError = 1143 + ErrTableaccessDenied = 1142 + ErrColumnaccessDenied = 1143 ErrIllegalGrantForTable = 1144 ErrGrantWrongHostOrUser = 1145 ErrNoSuchTable = 1146 ErrNonexistingTableGrant = 1147 ErrNotAllowedCommand = 1148 - ErrSyntaxError = 1149 + ErrSyntax = 1149 ErrDelayedCantChangeLock = 1150 ErrTooManyDelayedThreads = 1151 ErrAbortingConnection = 1152 ErrNetPacketTooLarge = 1153 ErrNetReadErrorFromPipe = 1154 - ErrNetFcntlError = 1155 + ErrNetFcntl = 1155 ErrNetPacketsOutOfOrder = 1156 - ErrNetUncompressError = 1157 - ErrNetReadError = 1158 + ErrNetUncompress = 1157 + ErrNetRead = 1158 ErrNetReadInterrupted = 1159 ErrNetErrorOnWrite = 1160 ErrNetWriteInterrupted = 1161 @@ -244,7 +244,7 @@ const ( ErrMixingNotAllowed = 1224 ErrDupArgument = 1225 ErrUserLimitReached = 1226 - ErrSpecificAccessDeniedError = 1227 + ErrSpecificAccessDenied = 1227 ErrLocalVariable = 1228 ErrGlobalVariable = 1229 ErrNoDefault = 1230 @@ -274,9 +274,9 @@ const ( ErrSlaveWasRunning = 1254 ErrSlaveWasNotRunning = 1255 ErrTooBigForUncompress = 1256 - ErrZlibZMemError = 1257 - ErrZlibZBufError = 1258 - ErrZlibZDataError = 1259 + ErrZlibZMem = 1257 + ErrZlibZBuf = 1258 + ErrZlibZData = 1259 ErrCutValueGroupConcat = 1260 ErrWarnTooFewRecords = 1261 ErrWarnTooManyRecords = 1262 @@ -387,7 +387,7 @@ const ( ErrIllegalValueForType = 1367 ErrViewNonupdCheck = 1368 ErrViewCheckFailed = 1369 - ErrProcaccessDeniedError = 1370 + ErrProcaccessDenied = 1370 ErrRelayLogFail = 1371 ErrPasswdLength = 1372 ErrUnknownTargetBinlog = 1373 @@ -496,50 +496,50 @@ const ( ErrForeignServerExists = 1476 ErrForeignServerDoesntExist = 1477 ErrIllegalHaCreateOption = 1478 - ErrPartitionRequiresValuesError = 1479 - ErrPartitionWrongValuesError = 1480 - ErrPartitionMaxvalueError = 1481 - ErrPartitionSubpartitionError = 1482 - ErrPartitionSubpartMixError = 1483 - ErrPartitionWrongNoPartError = 1484 - ErrPartitionWrongNoSubpartError = 1485 - ErrWrongExprInPartitionFuncError = 1486 - ErrNoConstExprInRangeOrListError = 1487 - ErrFieldNotFoundPartError = 1488 - ErrListOfFieldsOnlyInHashError = 1489 - ErrInconsistentPartitionInfoError = 1490 - ErrPartitionFuncNotAllowedError = 1491 - ErrPartitionsMustBeDefinedError = 1492 - ErrRangeNotIncreasingError = 1493 - ErrInconsistentTypeOfFunctionsError = 1494 - ErrMultipleDefConstInListPartError = 1495 - ErrPartitionEntryError = 1496 - ErrMixHandlerError = 1497 - ErrPartitionNotDefinedError = 1498 - ErrTooManyPartitionsError = 1499 - ErrSubpartitionError = 1500 + 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 - ErrBlobFieldInPartFuncError = 1502 + ErrBlobFieldInPartFunc = 1502 ErrUniqueKeyNeedAllFieldsInPf = 1503 - ErrNoPartsError = 1504 + ErrNoParts = 1504 ErrPartitionMgmtOnNonpartitioned = 1505 ErrForeignKeyOnPartitioned = 1506 ErrDropPartitionNonExistent = 1507 ErrDropLastPartition = 1508 ErrCoalesceOnlyOnHashPartition = 1509 ErrReorgHashOnlyOnSameNo = 1510 - ErrReorgNoParamError = 1511 + ErrReorgNoParam = 1511 ErrOnlyOnRangeListPartition = 1512 - ErrAddPartitionSubpartError = 1513 + ErrAddPartitionSubpart = 1513 ErrAddPartitionNoNewPartition = 1514 ErrCoalescePartitionNoPartition = 1515 ErrReorgPartitionNotExist = 1516 ErrSameNamePartition = 1517 - ErrNoBinlogError = 1518 + ErrNoBinlog = 1518 ErrConsecutiveReorgPartitions = 1519 ErrReorgOutsideRange = 1520 ErrPartitionFunctionFailure = 1521 - ErrPartStateError = 1522 + ErrPartState = 1522 ErrLimitedPartRange = 1523 ErrPluginIsNotLoaded = 1524 ErrWrongValue = 1525 @@ -547,9 +547,9 @@ const ( ErrFilegroupOptionOnlyOnce = 1527 ErrCreateFilegroupFailed = 1528 ErrDropFilegroupFailed = 1529 - ErrTablespaceAutoExtendError = 1530 + ErrTablespaceAutoExtend = 1530 ErrWrongSizeNumber = 1531 - ErrSizeOverflowError = 1532 + ErrSizeOverflow = 1532 ErrAlterFilegroupFailed = 1533 ErrBinlogRowLoggingFailed = 1534 ErrBinlogRowWrongTableDef = 1535 @@ -567,7 +567,7 @@ const ( ErrObsoleteColCountDoesntMatchCorrupted = 1547 ErrObsoleteCannotLoadFromTable = 1548 ErrEventCannotDelete = 1549 - ErrEventCompileError = 1550 + ErrEventCompile = 1550 ErrEventSameName = 1551 ErrEventDataTooLong = 1552 ErrDropIndexFk = 1553 @@ -580,21 +580,21 @@ const ( ErrStoredFunctionPreventsSwitchBinlogFormat = 1560 ErrNdbCantSwitchBinlogFormat = 1561 ErrPartitionNoTemporary = 1562 - ErrPartitionConstDomainError = 1563 + ErrPartitionConstDomain = 1563 ErrPartitionFunctionIsNotAllowed = 1564 - ErrDdlLogError = 1565 + ErrDdlLog = 1565 ErrNullInValuesLessThan = 1566 ErrWrongPartitionName = 1567 ErrCantChangeTxCharacteristics = 1568 ErrDupEntryAutoincrementCase = 1569 - ErrEventModifyQueueError = 1570 - ErrEventSetVarError = 1571 - ErrPartitionMergeError = 1572 + ErrEventModifyQueue = 1570 + ErrEventSetVar = 1571 + ErrPartitionMerge = 1572 ErrCantActivateLog = 1573 ErrRbrNotAvailable = 1574 - ErrBase64DecodeError = 1575 + ErrBase64Decode = 1575 ErrEventRecursionForbidden = 1576 - ErrEventsDbError = 1577 + ErrEventsDb = 1577 ErrOnlyIntegersAllowed = 1578 ErrUnsuportedLogEngine = 1579 ErrBadLogStatement = 1580 @@ -610,7 +610,7 @@ const ( ErrSlaveIncident = 1590 ErrNoPartitionForGivenValueSilent = 1591 ErrBinlogUnsafeStatement = 1592 - ErrSlaveFatalError = 1593 + ErrSlaveFatal = 1593 ErrSlaveRelayLogReadFailure = 1594 ErrSlaveRelayLogWriteFailure = 1595 ErrSlaveCreateEventFailure = 1596 @@ -642,9 +642,9 @@ const ( ErrWarnEngineTransactionRollback = 1622 ErrSlaveHeartbeatFailure = 1623 ErrSlaveHeartbeatValueOutOfRange = 1624 - ErrNdbReplicationSchemaError = 1625 - ErrConflictFnParseError = 1626 - ErrExceptionsWriteError = 1627 + ErrNdbReplicationSchema = 1625 + ErrConflictFnParse = 1626 + ErrExceptionsWrite = 1627 ErrTooLongTableComment = 1628 ErrTooLongFieldComment = 1629 ErrFuncInexistentNameCollision = 1630 @@ -670,12 +670,12 @@ const ( ErrSlaveIgnoreServerIds = 1650 ErrQueryCacheDisabled = 1651 ErrSameNamePartitionField = 1652 - ErrPartitionColumnListError = 1653 - ErrWrongTypeColumnValueError = 1654 - ErrTooManyPartitionFuncFieldsError = 1655 + ErrPartitionColumnList = 1653 + ErrWrongTypeColumnValue = 1654 + ErrTooManyPartitionFuncFields = 1655 ErrMaxvalueInValuesIn = 1656 - ErrTooManyValuesError = 1657 - ErrRowSinglePartitionFieldError = 1658 + ErrTooManyValues = 1657 + ErrRowSinglePartitionField = 1658 ErrFieldTypeNotAllowedAsPartitionField = 1659 ErrPartitionFieldsTooLong = 1660 ErrBinlogRowEngineAndStmtEngine = 1661 @@ -714,8 +714,8 @@ const ( ErrInsideTransactionPreventsSwitchSQLLogBin = 1694 ErrStoredFunctionPreventsSwitchSQLLogBin = 1695 ErrFailedReadFromParFile = 1696 - ErrValuesIsNotIntTypeError = 1697 - ErrAccessDeniedNoPasswordError = 1698 + ErrValuesIsNotIntType = 1697 + ErrAccessDeniedNoPassword = 1698 ErrSetPasswordAuthPlugin = 1699 ErrGrantPluginUserExists = 1700 ErrTruncateIllegalFk = 1701 @@ -827,13 +827,13 @@ const ( ErrDiscardFkChecksRunning = 1807 ErrTableSchemaMismatch = 1808 ErrTableInSystemTablespace = 1809 - ErrIoReadError = 1810 - ErrIoWriteError = 1811 + ErrIoRead = 1810 + ErrIoWrite = 1811 ErrTablespaceMissing = 1812 ErrTablespaceExists = 1813 ErrTablespaceDiscarded = 1814 - ErrInternalError = 1815 - ErrInnodbImportError = 1816 + ErrInternal = 1815 + ErrInnodbImport = 1816 ErrInnodbIndexCorrupt = 1817 ErrInvalidYearColumnLength = 1818 ErrNotValidPassword = 1819 diff --git a/mysqldef/errname.go b/mysqldef/errname.go index 7ecc731246..bc98fed16b 100644 --- a/mysqldef/errname.go +++ b/mysqldef/errname.go @@ -55,21 +55,21 @@ var MySQLErrName = map[uint16]string{ 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)", - ErrConCountError: "Too many connections", + 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", - ErrBadHostError: "Can't get hostname for your address", - ErrHandshakeError: "Bad handshake", - ErrDbaccessDeniedError: "Access denied for user '%-.48s'@'%-.64s' to database '%-.192s'", - ErrAccessDeniedError: "Access denied for user '%-.48s'@'%-.64s' (using password: %s)", - ErrNoDbError: "No database selected", - ErrUnknownComError: "Unknown command", - ErrBadNullError: "Column '%-.192s' cannot be null", - ErrBadDbError: "Unknown database '%-.192s'", - ErrTableExistsError: "Table '%-.192s' already exists", - ErrBadTableError: "Unknown table '%-.100s'", - ErrNonUniqError: "Column '%-.192s' in %-.192s is ambiguous", + 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", - ErrBadFieldError: "Unknown column '%-.192s' in '%-.192s'", + 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", @@ -79,7 +79,7 @@ var MySQLErrName = map[uint16]string{ ErrDupKeyname: "Duplicate key name '%-.192s'", ErrDupEntry: "Duplicate entry '%-.192s' for key %d", ErrWrongFieldSpec: "Incorrect column specifier for column '%-.192s'", - ErrParseError: "%s near '%-.80s' at line %d", + ErrParse: "%s near '%-.80s' at line %d", ErrEmptyQuery: "Query was empty", ErrNonuniqTable: "Not unique table/alias: '%-.192s'", ErrInvalidDefault: "Invalid default value for '%-.192s'", @@ -96,12 +96,12 @@ var MySQLErrName = map[uint16]string{ ErrGotSignal: "%s: Got signal %d. Aborting!\n", ErrShutdownComplete: "%s: Shutdown complete\n", ErrForcingClose: "%s: Forcing close of thread %ld user: '%-.48s'\n", - ErrIpsockError: "Can't create IP socket", + 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", - ErrFileExistsError: "File '%-.200s' already exists", + 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", @@ -110,7 +110,7 @@ var MySQLErrName = map[uint16]string{ 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", - ErrKillDeniedError: "You are not owner of thread %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", @@ -120,7 +120,7 @@ var MySQLErrName = map[uint16]string{ 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", - ErrUnknownError: "Unknown error", + ErrUnknown: "Unknown error", ErrUnknownProcedure: "Unknown procedure '%-.192s'", ErrWrongParamcountToProcedure: "Incorrect parameter count to procedure '%-.192s'", ErrWrongParametersToProcedure: "Incorrect parameters to procedure '%-.192s'", @@ -154,26 +154,26 @@ var MySQLErrName = map[uint16]string{ ErrWrongValueCountOnRow: "Column count doesn't match value count at row %ld", ErrCantReopenTable: "Can't reopen table: '%-.192s'", ErrInvalidUseOfNull: "Invalid use of NULL value", - ErrRegexpError: "Got error '%-.64s' from regexp", + 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'", - ErrTableaccessDeniedError: "%-.128s command denied to user '%-.48s'@'%-.64s' for table '%-.64s'", - ErrColumnaccessDeniedError: "%-.16s command denied to user '%-.48s'@'%-.64s' for column '%-.192s' in table '%-.192s'", + 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", - ErrSyntaxError: "You have an error in your SQL syntax; check the manual that corresponds to your MySQL server version for the right syntax to use", + 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", - ErrNetFcntlError: "Got an error from fcntl()", + ErrNetFcntl: "Got an error from fcntl()", ErrNetPacketsOutOfOrder: "Got packets out of order", - ErrNetUncompressError: "Couldn't uncompress communication packet", - ErrNetReadError: "Got an error reading communication packets", + 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", @@ -242,7 +242,7 @@ var MySQLErrName = map[uint16]string{ 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)", - ErrSpecificAccessDeniedError: "Access denied; you need (at least one of) the %-.128s privilege(s) for this operation", + 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", @@ -272,9 +272,9 @@ var MySQLErrName = map[uint16]string{ 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)", - ErrZlibZMemError: "ZLIB: Not enough memory", - ErrZlibZBufError: "ZLIB: Not enough room in the output buffer (probably, length of uncompressed data was corrupted)", - ErrZlibZDataError: "ZLIB: Input data 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", @@ -385,7 +385,7 @@ var MySQLErrName = map[uint16]string{ ErrIllegalValueForType: "Illegal %s '%-.192s' value found during parsing", ErrViewNonupdCheck: "CHECK OPTION on non-updatable view '%-.192s.%-.192s'", ErrViewCheckFailed: "CHECK OPTION failed '%-.192s.%-.192s'", - ErrProcaccessDeniedError: "%-.16s command denied to user '%-.48s'@'%-.64s' for routine '%-.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", @@ -445,11 +445,11 @@ var MySQLErrName = map[uint16]string{ 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 error: %-.64s", - ErrForeignDataSourceDoesntExist: "The foreign data source you are trying to reference does not exist. Data source error: %-.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 error: %-.64s", + 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", @@ -492,52 +492,52 @@ var MySQLErrName = map[uint16]string{ 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 error: %-.64s", + 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'", - ErrPartitionRequiresValuesError: "Syntax error: %-.64s PARTITIONING requires definition of VALUES %-.64s for each partition", - ErrPartitionWrongValuesError: "Only %-.64s PARTITIONING can use VALUES %-.64s in partition definition", - ErrPartitionMaxvalueError: "MAXVALUE can only be used in last partition definition", - ErrPartitionSubpartitionError: "Subpartitions can only be hash partitions and by key", - ErrPartitionSubpartMixError: "Must define subpartitions on all partitions if on one partition", - ErrPartitionWrongNoPartError: "Wrong number of partitions defined, mismatch with previous setting", - ErrPartitionWrongNoSubpartError: "Wrong number of subpartitions defined, mismatch with previous setting", - ErrWrongExprInPartitionFuncError: "Constant, random or timezone-dependent expressions in (sub)partitioning function are not allowed", - ErrNoConstExprInRangeOrListError: "Expression in RANGE/LIST VALUES must be constant", - ErrFieldNotFoundPartError: "Field in list of fields for partition function not found in table", - ErrListOfFieldsOnlyInHashError: "List of fields is only allowed in KEY partitions", - ErrInconsistentPartitionInfoError: "The partition info in the frm file is not consistent with what can be written into the frm file", - ErrPartitionFuncNotAllowedError: "The %-.192s function returns the wrong type", - ErrPartitionsMustBeDefinedError: "For %-.64s partitions each partition must be defined", - ErrRangeNotIncreasingError: "VALUES LESS THAN value must be strictly increasing for each partition", - ErrInconsistentTypeOfFunctionsError: "VALUES value must be of same type as partition function", - ErrMultipleDefConstInListPartError: "Multiple definition of same constant in list partitioning", - ErrPartitionEntryError: "Partitioning can not be used stand-alone in query", - ErrMixHandlerError: "The mix of handlers in the partitions is not allowed in this version of MySQL", - ErrPartitionNotDefinedError: "For the partitioned engine it is necessary to define all %-.64s", - ErrTooManyPartitionsError: "Too many partitions (including subpartitions) were defined", - ErrSubpartitionError: "It is only possible to mix RANGE/LIST partitioning with HASH/KEY partitioning for subpartitioning", + 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", - ErrBlobFieldInPartFuncError: "A BLOB field is not allowed in partition function", + ErrBlobFieldInPartFunc: "A BLOB field is not allowed in partition function", ErrUniqueKeyNeedAllFieldsInPf: "A %-.192s must include all columns in the table's partitioning function", - ErrNoPartsError: "Number of %-.64s = 0 is not an allowed value", + 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", - ErrReorgNoParamError: "REORGANIZE PARTITION without parameters can only be used on auto-partitioned tables using HASH PARTITIONs", + 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", - ErrAddPartitionSubpartError: "Trying to Add partition(s) with wrong number of subpartitions", + 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", - ErrNoBinlogError: "It is not allowed to shut off binlog on this command", + 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", - ErrPartStateError: "Partition state cannot be defined from CREATE/ALTER TABLE", + 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'", @@ -545,9 +545,9 @@ var MySQLErrName = map[uint16]string{ ErrFilegroupOptionOnlyOnce: "It is not allowed to specify %s more than once", ErrCreateFilegroupFailed: "Failed to create %s", ErrDropFilegroupFailed: "Failed to drop %s", - ErrTablespaceAutoExtendError: "The handler doesn't support autoextend of tablespaces", + ErrTablespaceAutoExtend: "The handler doesn't support autoextend of tablespaces", ErrWrongSizeNumber: "A size parameter was incorrectly specified, either number or on the form 10M", - ErrSizeOverflowError: "The size number was correct but we don't allow the digit part to be more than 2 billion", + 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", @@ -565,7 +565,7 @@ var MySQLErrName = map[uint16]string{ 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", - ErrEventCompileError: "Error during compilation of event's body", + 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", @@ -578,21 +578,21 @@ var MySQLErrName = map[uint16]string{ 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", - ErrPartitionConstDomainError: "Partition constant is out of partition function domain", + ErrPartitionConstDomain: "Partition constant is out of partition function domain", ErrPartitionFunctionIsNotAllowed: "This partition function is not allowed", - ErrDdlLogError: "Error in DDL log", + 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'", - ErrEventModifyQueueError: "Internal scheduler error %d", - ErrEventSetVarError: "Error during starting/stopping of the scheduler. Error code %u", - ErrPartitionMergeError: "Engine cannot be used in partitioned tables", + 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", - ErrBase64DecodeError: "Decoding of base64 string failed", + ErrBase64Decode: "Decoding of base64 string failed", ErrEventRecursionForbidden: "Recursion of EVENT DDL statements is forbidden when body is present", - ErrEventsDbError: "Cannot proceed because system tables used by Event Scheduler were found damaged at server start", + 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", @@ -608,7 +608,7 @@ var MySQLErrName = map[uint16]string{ 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", - ErrSlaveFatalError: "Fatal error: %s", + ErrSlaveFatal: "Fatal : %s", ErrSlaveRelayLogReadFailure: "Relay log read failure: %s", ErrSlaveRelayLogWriteFailure: "Relay log write failure: %s", ErrSlaveCreateEventFailure: "Failed to create %s", @@ -640,9 +640,9 @@ var MySQLErrName = map[uint16]string{ 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).", - ErrNdbReplicationSchemaError: "Bad schema for mysql.ndbReplication table. Message: %-.64s", - ErrConflictFnParseError: "Error in parsing conflict function. Message: %-.64s", - ErrExceptionsWriteError: "Write to exceptions table failed. Message: %-.128s\"", + 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", @@ -668,12 +668,12 @@ var MySQLErrName = map[uint16]string{ 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'", - ErrPartitionColumnListError: "Inconsistency in usage of column lists for partitioning", - ErrWrongTypeColumnValueError: "Partition column values of incorrect type", - ErrTooManyPartitionFuncFieldsError: "Too many fields in '%-.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", - ErrTooManyValuesError: "Cannot have more than one value for this type of %-.64s partitioning", - ErrRowSinglePartitionFieldError: "Row expressions in VALUES IN only allowed for multi-field column partitioning", + 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.", @@ -712,8 +712,8 @@ var MySQLErrName = map[uint16]string{ 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", - ErrValuesIsNotIntTypeError: "VALUES value for partition '%-.64s' must have type INT", - ErrAccessDeniedNoPasswordError: "Access denied for user '%-.48s'@'%-.64s'", + 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)", @@ -825,13 +825,13 @@ var MySQLErrName = map[uint16]string{ 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", - ErrIoReadError: "IO Read error: (%lu, %s) %s", - ErrIoWriteError: "IO Write error: (%lu, %s) %s", + 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'", - ErrInternalError: "Internal error: %s", - ErrInnodbImportError: "ALTER TABLE '%-.192s' IMPORT TABLESPACE failed with error %lu : '%s'", + 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", diff --git a/mysqldef/error_test.go b/mysqldef/error_test.go index d08c104ed3..795111016f 100644 --- a/mysqldef/error_test.go +++ b/mysqldef/error_test.go @@ -23,13 +23,13 @@ type testSQLErrorSuite struct { } func (s *testSQLErrorSuite) TestSQLError(c *C) { - e := NewError(ErrNoDbError, "no db error") + e := NewError(ErrNoDb, "no db error") c.Assert(len(e.Error()), Greater, 0) e = NewError(0, "customized error") c.Assert(len(e.Error()), Greater, 0) - e = NewDefaultError(ErrNoDbError) + e = NewDefaultError(ErrNoDb) c.Assert(len(e.Error()), Greater, 0) e = NewDefaultError(0, "customized error") diff --git a/mysqldef/state.go b/mysqldef/state.go index 7d581c1ac4..9dfb4e9c14 100644 --- a/mysqldef/state.go +++ b/mysqldef/state.go @@ -24,20 +24,20 @@ var MySQLState = map[uint16]string{ ErrDupKey: "23000", ErrOutofmemory: "HY001", ErrOutOfSortmemory: "HY001", - ErrConCountError: "08004", - ErrBadHostError: "08S01", - ErrHandshakeError: "08S01", - ErrDbaccessDeniedError: "42000", - ErrAccessDeniedError: "28000", - ErrNoDbError: "3D000", - ErrUnknownComError: "08S01", - ErrBadNullError: "23000", - ErrBadDbError: "42000", - ErrTableExistsError: "42S01", - ErrBadTableError: "42S02", - ErrNonUniqError: "23000", + 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", - ErrBadFieldError: "42S22", + ErrBadField: "42S22", ErrWrongFieldWithGroup: "42000", ErrWrongSumSelect: "42000", ErrWrongGroupField: "42000", @@ -47,7 +47,7 @@ var MySQLState = map[uint16]string{ ErrDupKeyname: "42000", ErrDupEntry: "23000", ErrWrongFieldSpec: "42000", - ErrParseError: "42000", + ErrParse: "42000", ErrEmptyQuery: "42000", ErrNonuniqTable: "42000", ErrInvalidDefault: "42000", @@ -60,7 +60,7 @@ var MySQLState = map[uint16]string{ ErrTooBigFieldlength: "42000", ErrWrongAutoKey: "42000", ErrForcingClose: "08S01", - ErrIpsockError: "08S01", + ErrIpsock: "08S01", ErrNoSuchIndex: "42S12", ErrWrongFieldTerminators: "42000", ErrBlobsAndNoTerminated: "42000", @@ -85,24 +85,24 @@ var MySQLState = map[uint16]string{ ErrPasswordNoMatch: "42000", ErrWrongValueCountOnRow: "21S01", ErrInvalidUseOfNull: "22004", - ErrRegexpError: "42000", + ErrRegexp: "42000", ErrMixOfGroupFuncAndFields: "42000", ErrNonexistingGrant: "42000", - ErrTableaccessDeniedError: "42000", - ErrColumnaccessDeniedError: "42000", + ErrTableaccessDenied: "42000", + ErrColumnaccessDenied: "42000", ErrIllegalGrantForTable: "42000", ErrGrantWrongHostOrUser: "42000", ErrNoSuchTable: "42S02", ErrNonexistingTableGrant: "42000", ErrNotAllowedCommand: "42000", - ErrSyntaxError: "42000", + ErrSyntax: "42000", ErrAbortingConnection: "08S01", ErrNetPacketTooLarge: "08S01", ErrNetReadErrorFromPipe: "08S01", - ErrNetFcntlError: "08S01", + ErrNetFcntl: "08S01", ErrNetPacketsOutOfOrder: "08S01", - ErrNetUncompressError: "08S01", - ErrNetReadError: "08S01", + ErrNetUncompress: "08S01", + ErrNetRead: "08S01", ErrNetReadInterrupted: "08S01", ErrNetErrorOnWrite: "08S01", ErrNetWriteInterrupted: "08S01", @@ -132,7 +132,7 @@ var MySQLState = map[uint16]string{ ErrConnectToMaster: "08S01", ErrWrongNumberOfColumnsInSelect: "21000", ErrUserLimitReached: "42000", - ErrSpecificAccessDeniedError: "42000", + ErrSpecificAccessDenied: "42000", ErrNoDefault: "42000", ErrWrongValueForVar: "42000", ErrWrongTypeForVar: "42000", @@ -192,7 +192,7 @@ var MySQLState = map[uint16]string{ ErrSpCaseNotFound: "20000", ErrDivisionByZero: "22012", ErrIllegalValueForType: "22007", - ErrProcaccessDeniedError: "42000", + ErrProcaccessDenied: "42000", ErrXaerNota: "XAE04", ErrXaerInval: "XAE05", ErrXaerRmfail: "XAE07", @@ -237,7 +237,7 @@ var MySQLState = map[uint16]string{ ErrResignalWithoutActiveHandler: "0K000", ErrSpatialMustHaveGeomCol: "42000", ErrDataOutOfRange: "22003", - ErrAccessDeniedNoPasswordError: "28000", + ErrAccessDeniedNoPassword: "28000", ErrTruncateIllegalFk: "42000", ErrDaInvalidConditionNumber: "35000", ErrForeignDuplicateKeyWithChildInfo: "23000", diff --git a/tidb-server/server/conn.go b/tidb-server/server/conn.go index 1083137969..9b0215e950 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.ErrAccessDeniedError, cc.user, addr, "Yes")) + return errors.Trace(mysql.NewDefaultError(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.ErrAccessDeniedError, cc.user, host, "Yes")) + return errors.Trace(mysql.NewDefaultError(mysql.ErrAccessDenied, cc.user, host, "Yes")) } } return nil @@ -284,7 +284,7 @@ func (cc *clientConn) dispatch(data []byte) error { return cc.handleStmtReset(data) default: msg := fmt.Sprintf("command %d not supported now", cmd) - return mysql.NewError(mysql.ErrUnknownError, msg) + return mysql.NewError(mysql.ErrUnknown, msg) } } @@ -324,7 +324,7 @@ func (cc *clientConn) writeError(e error) error { var ok bool originErr := errors.Cause(e) if m, ok = originErr.(*mysql.SQLError); !ok { - m = mysql.NewError(mysql.ErrUnknownError, e.Error()) + m = mysql.NewError(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 a40ee4a1fe..55816a720b 100644 --- a/tidb-server/server/conn_stmt.go +++ b/tidb-server/server/conn_stmt.go @@ -121,7 +121,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.ErrUnknownError, fmt.Sprintf("unsupported flag %d", flag)) + return mysql.NewError(mysql.ErrUnknown, fmt.Sprintf("unsupported flag %d", flag)) } //skip iteration-count, always 1 From f994b8b82a2e2f476d482a5d3a2b97a11c75e0ec Mon Sep 17 00:00:00 2001 From: siddontang Date: Wed, 14 Oct 2015 17:22:49 +0800 Subject: [PATCH 22/58] memkv: reduce test time --- kv/memkv/btree_test.go | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) 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) From 73d3f1b84eb7f315469b4cb77846304efa8162d6 Mon Sep 17 00:00:00 2001 From: ngaut Date: Wed, 14 Oct 2015 17:33:58 +0800 Subject: [PATCH 23/58] sessionctx: Rename sessionctx_test.go to domainctx_test.go Currently sessionctx_test.go is test file for domainctx --- sessionctx/sessionctx_test.go | 44 ----------------------------------- 1 file changed, 44 deletions(-) delete mode 100644 sessionctx/sessionctx_test.go diff --git a/sessionctx/sessionctx_test.go b/sessionctx/sessionctx_test.go deleted file mode 100644 index a4d6e7fe9b..0000000000 --- a/sessionctx/sessionctx_test.go +++ /dev/null @@ -1,44 +0,0 @@ -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package sessionctx - -import ( - "testing" - - . "github.com/pingcap/check" - "github.com/pingcap/tidb/util/mock" -) - -func TestT(t *testing.T) { - TestingT(t) -} - -var _ = Suite(&testSessionCtxSuite{}) - -type testSessionCtxSuite struct { -} - -func (s *testSessionCtxSuite) TestDomain(c *C) { - ctx := mock.NewContext() - - c.Assert(domainKey.String(), Not(Equals), "") - - BindDomain(ctx, nil) - v := GetDomain(ctx) - c.Assert(v, IsNil) - - ctx.ClearValue(domainKey) - v = GetDomain(ctx) - c.Assert(v, IsNil) -} From 4d1532c0b820d6d9287e6cd080e2ef43b2c4dd7a Mon Sep 17 00:00:00 2001 From: ngaut Date: Wed, 14 Oct 2015 17:37:05 +0800 Subject: [PATCH 24/58] Add missing file --- sessionctx/domainctx_test.go | 44 ++++++++++++++++++++++++++++++++++++ 1 file changed, 44 insertions(+) create mode 100644 sessionctx/domainctx_test.go diff --git a/sessionctx/domainctx_test.go b/sessionctx/domainctx_test.go new file mode 100644 index 0000000000..a4d6e7fe9b --- /dev/null +++ b/sessionctx/domainctx_test.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 sessionctx + +import ( + "testing" + + . "github.com/pingcap/check" + "github.com/pingcap/tidb/util/mock" +) + +func TestT(t *testing.T) { + TestingT(t) +} + +var _ = Suite(&testSessionCtxSuite{}) + +type testSessionCtxSuite struct { +} + +func (s *testSessionCtxSuite) TestDomain(c *C) { + ctx := mock.NewContext() + + c.Assert(domainKey.String(), Not(Equals), "") + + BindDomain(ctx, nil) + v := GetDomain(ctx) + c.Assert(v, IsNil) + + ctx.ClearValue(domainKey) + v = GetDomain(ctx) + c.Assert(v, IsNil) +} From a53c05b6f457f282092622bc6dbe6a743fd4390a Mon Sep 17 00:00:00 2001 From: siddontang Date: Wed, 14 Oct 2015 19:14:17 +0800 Subject: [PATCH 25/58] *: simplify generating mysql error function --- mysqldef/error.go | 11 +++++------ mysqldef/error_test.go | 8 ++++---- tidb-server/server/conn.go | 9 ++++----- tidb-server/server/conn_stmt.go | 9 ++++----- tidb-server/server/driver_tidb.go | 2 +- 5 files changed, 18 insertions(+), 21 deletions(-) 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 795111016f..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(ErrNoDb, "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(ErrNoDb) + 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/tidb-server/server/conn.go b/tidb-server/server/conn.go index 9b0215e950..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.ErrAccessDenied, 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.ErrAccessDenied, 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.ErrUnknown, msg) + return mysql.NewErrf(mysql.ErrUnknown, "command %d not supported now", cmd) } } @@ -324,7 +323,7 @@ func (cc *clientConn) writeError(e error) error { var ok bool originErr := errors.Cause(e) if m, ok = originErr.(*mysql.SQLError); !ok { - m = mysql.NewError(mysql.ErrUnknown, e.Error()) + 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 55816a720b..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.ErrUnknownStmtHandler, + 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.ErrUnknown, 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.ErrUnknownStmtHandler, + 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.ErrUnknownStmtHandler, + 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 38f2ba3a7d..79c23ae93c 100644 --- a/tidb-server/server/driver_tidb.go +++ b/tidb-server/server/driver_tidb.go @@ -79,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.ErrWrongArguments, "stmt_send_longdata") + return mysql.NewErr(mysql.ErrWrongArguments, "stmt_send_longdata") } ts.boundParams[paramID] = append(ts.boundParams[paramID], data...) return nil From 836b254f819224407146825af75e8aec4d68048f Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Wed, 14 Oct 2015 19:47:01 +0800 Subject: [PATCH 26/58] ast: add more statements. --- ast/ast.go | 33 ++- ast/base.go | 88 ++++++ ast/ddl.go | 502 +++++++++++++++++++++++++------- ast/dml.go | 271 +++++++++--------- ast/expressions.go | 698 +++++++++++++-------------------------------- ast/functions.go | 251 ++++++++++++++++ ast/misc.go | 203 ++++++++++--- 7 files changed, 1280 insertions(+), 766 deletions(-) create mode 100644 ast/base.go create mode 100644 ast/functions.go diff --git a/ast/ast.go b/ast/ast.go index ddcb37a38f..a857aa72de 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -20,6 +20,7 @@ import ( ) // 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. @@ -31,9 +32,10 @@ type Node interface { SetText(text string) } -// Expression is a node that can be evaluated. -type Expression interface { - // Node is embeded in Expression. +// 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 @@ -43,6 +45,31 @@ type Expression interface { 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. diff --git a/ast/base.go b/ast/base.go new file mode 100644 index 0000000000..17d63d164e --- /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 { + txt string +} + +// SetText implements Node interface. +func (n *node) SetText(text string) { + n.txt = text +} + +// Text implements Node interface. +func (n *node) Text() string { + return n.txt +} + +// 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 index 0ab0bf82c3..a9bc7b28bd 100644 --- a/ast/ddl.go +++ b/ast/ddl.go @@ -1,3 +1,16 @@ +// 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 ( @@ -5,22 +18,21 @@ import ( ) var ( - _ Node = &CreateDatabaseStmt{} - _ Node = &DropDatabaseStmt{} - _ Node = &IndexColName{} - _ Node = &ReferenceDef{} - _ Node = &ConstraintOpt{} - _ Node = &TableConstraint{} - _ Node = &ColumnDef{} - _ Node = &CreateTableOption{} - _ Node = &CreateTableStmt{} - _ Node = &DropTableStmt{} - _ Node = &CreateIndexStmt{} - _ Node = &DropTableStmt{} - _ Node = &AlterTableOpt{} - _ Node = &ColumnPosition{} - _ Node = &AlterSpecification{} - _ Node = &AlterTableStmt{} + _ 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. @@ -32,83 +44,154 @@ type CharsetOpt struct { // CreateDatabaseStmt is a statement to create a database. // See: https://dev.mysql.com/doc/refman/5.7/en/create-database.html type CreateDatabaseStmt struct { - txtNode + 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 { - txtNode + 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 { - Column *ColumnRef + 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 { - TableIdent *TableRef + node + + Table *TableRef IndexColNames []*IndexColName } -// Constraints. -const ( - ConstrNoConstr = iota - ConstrPrimaryKey - ConstrForeignKey - ConstrNotNull - ConstrAutoIncrement - ConstrDefaultValue - ConstrUniq - ConstrIndex - ConstrUniqIndex - ConstrKey - ConstrUniqKey - ConstrNull - ConstrOnUpdate - ConstrFulltext - ConstrComment -) - -// ConstraintOpt is used for parsing column constraint info from SQL. -type ConstraintOpt struct { - Tp int - Bvalue bool - Evalue Expression +// 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) } -// Table Options. +// ColumnOptionType is the type for ColumnOption. +type ColumnOptionType int + +// ColumnOption types. const ( - TblOptNone = iota - TblOptEngine - TblOptCharset - TblOptCollate - TblOptAutoIncrement - TblOptComment - TblOptAvgRowLength - TblOptCheckSum - TblOptCompression - TblOptConnection - TblOptPassword - TblOptKeyBlockSize - TblOptMaxRows - TblOptMinRows + ColumnOptionNoOption ColumnOptionType = iota + ColumnOptionPrimaryKey + ColumnOptionNotNull + ColumnOptionAutoIncrement + ColumnOptionDefaultValue + ColumnOptionUniq + ColumnOptionIndex + ColumnOptionUniqIndex + ColumnOptionKey + ColumnOptionUniqKey + ColumnOptionNull + ColumnOptionOnUpdate // For Timestamp and Datetime only. + ColumnOptionFulltext + ColumnOptionComment ) -// TableConstraint is constraint for table definition. -type TableConstraint struct { - Tp int - ConstrName string +// 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 @@ -117,46 +200,119 @@ type TableConstraint struct { Refer *ReferenceDef } -// ColumnDef is used for parsing column definition from SQL. -type ColumnDef struct { - Name string - Tp *types.FieldType - Constraints []*ConstraintOpt +// 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) } -// CreateTableOption is the collection of table options. -type CreateTableOption struct { - Engine string - Charset string - Collate string - AutoIncrement uint64 +// 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 { - txtNode + ddlNode IfNotExists bool - Ident TableIdent + Table *TableRef Cols []*ColumnDef - Constraints []*TableConstraint - Opt *CreateTableOption + 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 { - txtNode + 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 { - txtNode + ddlNode IndexName string Table *TableRef @@ -164,52 +320,206 @@ type CreateIndexStmt struct { 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 - - Text string } -// AlterTableOpt is used for parsing table option from SQL. -type AlterTableOpt struct { - Tp int +// 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 int = iota + ColumnPositionNone ColumnPositionType = iota ColumnPositionFirst ColumnPositionAfter ) // ColumnPosition represent the position of the newly added column type ColumnPosition struct { + node // ColumnPositionNone | ColumnPositionFirst | ColumnPositionAfter - Type int + Tp ColumnPositionType // RelativeColumn is the column the newly added column after if type is ColumnPositionAfter - RelativeColumn *ColumnRef + RelativeColumn *ColumnRefExpr } -// AlterSpecification alter table specification -type AlterSpecification struct { - Action int +// 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 *TableConstraint - TableOpts []*AlterTableOpt + 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 { - txtNode + ddlNode - Ident TableIdent - Specs []*AlterSpecification + 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 index aa30c22fec..9c1074cb13 100644 --- a/ast/dml.go +++ b/ast/dml.go @@ -18,34 +18,17 @@ import ( ) var ( - _ Node = &JoinNode{} - _ Node = &TableRef{} - _ Node = &TableSource{} - _ Node = &SelectNode{} - _ Node = &Assignment{} - _ Node = &InsertIntoStmt{} - _ Node = &DeleteStmt{} - _ Node = &UpdateStmt{} - _ Node = &TruncateTableStmt{} - _ Node = &UnionStmt{} + _ DMLNode = &InsertStmt{} + _ DMLNode = &DeleteStmt{} + _ DMLNode = &UpdateStmt{} + _ DMLNode = &SelectStmt{} + _ Node = &Join{} + _ Node = &Union{} + _ Node = &TableRef{} + _ Node = &TableSource{} + _ Node = &Assignment{} ) -// txtNode is the struct implements partial node interface. -// can be embeded by other nodes. -type txtNode struct { - txt string -} - -// SetText implements Node interface. -func (bn *txtNode) SetText(text string) { - bn.txt = text -} - -// Text implements Node interface. -func (bn *txtNode) Text() string { - return bn.txt -} - // JoinType is join type, including cross/left/right/full. type JoinType int @@ -58,9 +41,9 @@ const ( RightJoin ) -// JoinNode represents table join. -type JoinNode struct { - txtNode +// Join represents table join. +type Join struct { + node // Left table can be TableSource or JoinNode. Left Node @@ -71,33 +54,31 @@ type JoinNode struct { } // Accept implements Node Accept interface. -func (jn *JoinNode) Accept(v Visitor) (Node, bool) { - if !v.Enter(jn) { - return jn, false +func (j *Join) Accept(v Visitor) (Node, bool) { + if !v.Enter(j) { + return j, false } - node, ok := jn.Left.Accept(v) + node, ok := j.Left.Accept(v) if !ok { - return jn, false + return j, false } - jn.Left = node - node, ok = jn.Right.Accept(v) - if !ok { - return jn, false + j.Left = node + if j.Right != nil { + node, ok = j.Right.Accept(v) + if !ok { + return j, false + } + j.Right = node } - jn.Right = node - return v.Leave(jn) -} - -type TableIdent struct { - Schema model.CIStr - Name model.CIStr + return v.Leave(j) } // TableRef represents a reference to actual table. type TableRef struct { - txtNode + node - Ident TableIdent + Schema model.CIStr + Name model.CIStr } // Accept implements Node Accept interface. @@ -110,7 +91,7 @@ func (tr *TableRef) Accept(v Visitor) (Node, bool) { // TableSource represents table source with a name. type TableSource struct { - txtNode + node // Source is the source of the data, can be a TableRef, // a SubQuery, or a JoinNode. @@ -133,44 +114,66 @@ func (ts *TableSource) Accept(v Visitor) (Node, bool) { return v.Leave(ts) } -// LockType is select lock type. -type LockType int +// Union represents union select statement. +type Union struct { + node -// Select Lock Type. + 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 LockType = iota + SelectLockNone SelectLockType = iota SelectLockForUpdate SelectLockInShareMode ) -// SelectNode represents the select query node. -type SelectNode struct { - txtNode +// 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 []Expression + Fields []ExprNode // From is the from clause of the query. - From *JoinNode + From *Join // Where is the where clause in select statement. - Where Expression + Where ExprNode // GroupBy is the group by expression list. - GroupBy []Expression + GroupBy []ExprNode // Having is the having condition. - Having Expression + Having ExprNode // OrderBy is the odering expression list. - OrderBy []Expression + OrderBy []ExprNode // Offset is the offset value. Offset int // Limit is the limit value. Limit int // Lock is the lock type - LockTp LockType + LockTp SelectLockType + // Unions is the union select statement. + Unions []*Union } // Accept implements Node Accept interface. -func (sn *SelectNode) Accept(v Visitor) (Node, bool) { +func (sn *SelectStmt) Accept(v Visitor) (Node, bool) { if !v.Enter(sn) { return sn, false } @@ -179,52 +182,64 @@ func (sn *SelectNode) Accept(v Visitor) (Node, bool) { if !ok { return sn, false } - sn.Fields[i] = node.(Expression) + sn.Fields[i] = node.(ExprNode) } - node, ok := sn.From.Accept(v) - if !ok { - return sn, false + if sn.From != nil { + node, ok := sn.From.Accept(v) + if !ok { + return sn, false + } + sn.From = node.(*Join) } - sn.From = node.(*JoinNode) - node, ok = sn.Where.Accept(v) - if !ok { - return sn, false + if sn.Where != nil { + node, ok := sn.Where.Accept(v) + if !ok { + return sn, false + } + sn.Where = node.(ExprNode) } - sn.Where = node.(Expression) for i, val := range sn.GroupBy { - node, ok = val.Accept(v) + node, ok := val.Accept(v) if !ok { return sn, false } - sn.GroupBy[i] = node.(Expression) + sn.GroupBy[i] = node.(ExprNode) } - - node, ok = sn.Having.Accept(v) - if !ok { - return sn, false + if sn.Having != nil { + node, ok := sn.Having.Accept(v) + if !ok { + return sn, false + } + sn.Having = node.(ExprNode) } - sn.Having = node.(Expression) for i, val := range sn.OrderBy { - node, ok = val.Accept(v) + node, ok := val.Accept(v) if !ok { return sn, false } - sn.OrderBy[i] = node.(Expression) + 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 { - txtNode + node // Column is the column reference to be assigned. - Column *ColumnRef + Column *ColumnRefExpr // Expr is the expression assigning to ColName. - Expr Expression + Expr ExprNode } // Accept implements Node Accept interface. @@ -236,22 +251,22 @@ func (as *Assignment) Accept(v Visitor) (Node, bool) { if !ok { return as, false } - as.Column = node.(*ColumnRef) + as.Column = node.(*ColumnRefExpr) node, ok = as.Expr.Accept(v) if !ok { return as, false } - as.Expr = node.(Expression) + as.Expr = node.(ExprNode) return v.Leave(as) } -// InsertIntoStmt is a statement to insert new rows into an existing table. +// 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 InsertIntoStmt struct { - txtNode +type InsertStmt struct { + dmlNode - Columns []*ColumnRef - Lists [][]Expression + Columns []*ColumnRefExpr + Lists [][]ExprNode Table *TableRef Setlist []*Assignment Priority int @@ -259,7 +274,7 @@ type InsertIntoStmt struct { } // Accept implements Node Accept interface. -func (in *InsertIntoStmt) Accept(v Visitor) (Node, bool) { +func (in *InsertStmt) Accept(v Visitor) (Node, bool) { if !v.Enter(in) { return in, false } @@ -268,7 +283,7 @@ func (in *InsertIntoStmt) Accept(v Visitor) (Node, bool) { if !ok { return in, false } - in.Columns[i] = node.(*ColumnRef) + in.Columns[i] = node.(*ColumnRefExpr) } for i, list := range in.Lists { for j, val := range list { @@ -276,7 +291,7 @@ func (in *InsertIntoStmt) Accept(v Visitor) (Node, bool) { if !ok { return in, false } - in.Lists[i][j] = node.(Expression) + in.Lists[i][j] = node.(ExprNode) } } for i, val := range in.Setlist { @@ -299,11 +314,11 @@ func (in *InsertIntoStmt) Accept(v Visitor) (Node, bool) { // DeleteStmt is a statement to delete rows from table. // See: https://dev.mysql.com/doc/refman/5.7/en/delete.html type DeleteStmt struct { - txtNode + dmlNode Tables []*TableRef - Where Expression - Order []Expression + Where ExprNode + Order []ExprNode Limit int LowPriority bool Ignore bool @@ -325,17 +340,20 @@ func (de *DeleteStmt) Accept(v Visitor) (Node, bool) { de.Tables[i] = node.(*TableRef) } - node, ok := de.Where.Accept(v) - if !ok { - return de, false - } - de.Where = node.(Expression) - for i, val := range de.Order { - node, ok = val.Accept(v) + if de.Where != nil { + node, ok := de.Where.Accept(v) if !ok { return de, false } - de.Order[i] = node.(Expression) + 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) } @@ -343,12 +361,12 @@ func (de *DeleteStmt) Accept(v Visitor) (Node, bool) { // 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 { - txtNode + dmlNode - TableRefs *JoinNode + TableRefs *Join List []*Assignment - Where Expression - Order []Expression + Where ExprNode + Order []ExprNode Limit int LowPriority bool Ignore bool @@ -364,7 +382,7 @@ func (up *UpdateStmt) Accept(v Visitor) (Node, bool) { if !ok { return up, false } - up.TableRefs = node.(*JoinNode) + up.TableRefs = node.(*Join) for i, val := range up.List { node, ok = val.Accept(v) if !ok { @@ -372,37 +390,20 @@ func (up *UpdateStmt) Accept(v Visitor) (Node, bool) { } up.List[i] = node.(*Assignment) } - node, ok = up.Where.Accept(v) - if !ok { - return up, false + if up.Where != nil { + node, ok = up.Where.Accept(v) + if !ok { + return up, false + } + up.Where = node.(ExprNode) } - up.Where = node.(Expression) + for i, val := range up.Order { node, ok = val.Accept(v) if !ok { return up, false } - up.Order[i] = node.(Expression) + up.Order[i] = node.(ExprNode) } return v.Leave(up) } - -// 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 { - txtNode - - Table *TableRef -} - -// UnionStmt is a statement to combine results from multiple SelectStmts. -// See: https://dev.mysql.com/doc/refman/5.7/en/union.html -type UnionStmt struct { - txtNode - - Distincts []bool - Selects []*SelectNode - Limit int - Offset int - OrderBy []Expression -} diff --git a/ast/expressions.go b/ast/expressions.go index 8ace5f750f..d717561cde 100644 --- a/ast/expressions.go +++ b/ast/expressions.go @@ -14,103 +14,70 @@ package ast import ( - "github.com/pingcap/tidb/expression/builtin" "github.com/pingcap/tidb/model" "github.com/pingcap/tidb/parser/opcode" - "github.com/pingcap/tidb/util/types" - "strings" ) var ( - _ Expression = &Value{} - _ Expression = &Between{} - _ Expression = &BinaryOperation{} - _ Expression = &Call{} - _ Expression = &WhenClause{} - _ Expression = &FunctionCase{} - _ Expression = &FunctionCast{} - _ Expression = &SubQuery{} - _ Expression = &CompareSubQuery{} - _ Expression = &ColumnRef{} - _ Expression = &FunctionConvert{} - _ Expression = &Default{} - _ Expression = &ExistsSubQuery{} - _ Expression = &Extract{} - _ Expression = &PatternIn{} - _ Expression = &IsNull{} - _ Expression = &IsTruth{} - _ Expression = &PatternLike{} - _ Expression = &ParamMarker{} - _ Expression = &PExpr{} - _ Expression = &Position{} - _ Expression = &PatternRegexp{} - _ Expression = &Row{} - _ Expression = &FunctionSubstring{} - _ Expression = &FunctionTrim{} - _ Expression = &UnaryOperation{} - _ Expression = &Values{} - _ Expression = &Variable{} + _ 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{} ) -// tpNode is the struct implements partial expression interface. -// can be embeded by other nodes. -type tpNode struct { - tp *types.FieldType -} - -// SetType implements Expression interface. -func (tn *tpNode) SetType(tp *types.FieldType) { - tn.tp = tp -} - -// GetType implements Expression interface. -func (tn *tpNode) GetType() *types.FieldType { - return tn.tp -} - -type nonStatic struct{} - -func (ns *nonStatic) IsStatic() bool { - return false -} - -// Value is the simple value expression. -type Value struct { - txtNode - tpNode +// ValueExpr is the simple value expression. +type ValueExpr struct { + exprNode // Val is the literal value. Val interface{} } -// IsStatic implements Expression. -func (val *Value) IsStatic() bool { +// IsStatic implements ExprNode interface. +func (val *ValueExpr) IsStatic() bool { return true } // Accept implements Node interface. -func (val *Value) Accept(v Visitor) (Node, bool) { +func (val *ValueExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(val) { return val, false } return v.Leave(val) } -// Between is for "between and" or "not between and" expression. -type Between struct { - txtNode - tpNode +// BetweenExpr is for "between and" or "not between and" expression. +type BetweenExpr struct { + exprNode // Expr is the expression to be checked. - Expr Expression + Expr ExprNode // Left is the expression for minimal value in the range. - Left Expression + Left ExprNode // Right is the expression for maximum value in the range. - Right Expression + Right ExprNode // Not is true, the expression is "not between and". Not bool } // Accept implements Node interface. -func (b *Between) Accept(v Visitor) (Node, bool) { +func (b *BetweenExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(b) { return b, false } @@ -119,42 +86,41 @@ func (b *Between) Accept(v Visitor) (Node, bool) { if !ok { return b, false } - b.Expr = node.(Expression) + b.Expr = node.(ExprNode) node, ok = b.Left.Accept(v) if !ok { return b, false } - b.Left = node.(Expression) + b.Left = node.(ExprNode) node, ok = b.Right.Accept(v) if !ok { return b, false } - b.Right = node.(Expression) + b.Right = node.(ExprNode) return v.Leave(b) } -// IsStatic implements the Expression IsStatic interface. -func (b *Between) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (b *BetweenExpr) IsStatic() bool { return b.Expr.IsStatic() && b.Left.IsStatic() && b.Right.IsStatic() } -// BinaryOperation is for binary operation like 1 + 1, 1 - 1, etc. -type BinaryOperation struct { - txtNode - tpNode +// 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 Expression + L ExprNode // R is the right expression in BinaryOperation. - R Expression + R ExprNode } // Accept implements Node interface. -func (o *BinaryOperation) Accept(v Visitor) (Node, bool) { +func (o *BinaryOperationExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(o) { return o, false } @@ -163,73 +129,29 @@ func (o *BinaryOperation) Accept(v Visitor) (Node, bool) { if !ok { return o, false } - o.L = node.(Expression) + o.L = node.(ExprNode) node, ok = o.R.Accept(v) if !ok { return o, false } - o.R = node.(Expression) + o.R = node.(ExprNode) return v.Leave(o) } -// IsStatic implements the Expression IsStatic interface. -func (o *BinaryOperation) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (o *BinaryOperationExpr) IsStatic() bool { return o.L.IsStatic() && o.R.IsStatic() } -// Call is for function expression. -type Call struct { - txtNode - tpNode - // F is the function name. - F string - // Args is the function args. - Args []Expression - // Distinct only affetcts sum, avg, count, group_concat, - // so we can ignore it in other functions - Distinct bool -} - -// Accept implements Node interface. -func (c *Call) 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.(Expression) - } - return v.Leave(c) -} - -// IsStatic implements the Expression IsStatic interface. -func (c *Call) 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 -} - -// WhenClause is the expression in Case expression for "when condition then result". +// WhenClause is the when clause in Case expression for "when condition then result". type WhenClause struct { - txtNode - tpNode + node // Expr is the condition expression in WhenClause. - Expr Expression + Expr ExprNode // Result is the result expression in WhenClause. - Result Expression + Result ExprNode } // Accept implements Node Accept interface. @@ -241,60 +163,63 @@ func (w *WhenClause) Accept(v Visitor) (Node, bool) { if !ok { return w, false } - w.Expr = node.(Expression) + w.Expr = node.(ExprNode) node, ok = w.Result.Accept(v) if !ok { return w, false } - w.Result = node.(Expression) + w.Result = node.(ExprNode) return v.Leave(w) } -// IsStatic implements the Expression IsStatic interface. +// IsStatic implements the ExprNode IsStatic interface. func (w *WhenClause) IsStatic() bool { return w.Expr.IsStatic() && w.Result.IsStatic() } -// FunctionCase is the case expression. -type FunctionCase struct { - txtNode - tpNode +// CaseExpr is the case expression. +type CaseExpr struct { + exprNode // Value is the compare value expression. - Value Expression + Value ExprNode // WhenClauses is the condition check expression. WhenClauses []*WhenClause // ElseClause is the else result expression. - ElseClause Expression + ElseClause ExprNode } // Accept implements Node Accept interface. -func (f *FunctionCase) Accept(v Visitor) (Node, bool) { +func (f *CaseExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(f) { return f, false } - node, ok := f.Value.Accept(v) - if !ok { - return f, false + if f.Value != nil { + node, ok := f.Value.Accept(v) + if !ok { + return f, false + } + f.Value = node.(ExprNode) } - f.Value = node.(Expression) for i, val := range f.WhenClauses { - node, ok = val.Accept(v) + node, ok := val.Accept(v) if !ok { return f, false } f.WhenClauses[i] = node.(*WhenClause) } - node, ok = f.ElseClause.Accept(v) - if !ok { - return f, false + if f.ElseClause != nil { + node, ok := f.ElseClause.Accept(v) + if !ok { + return f, false + } + f.ElseClause = node.(ExprNode) } - f.ElseClause = node.(Expression) return v.Leave(f) } -// IsStatic implements the Expression IsStatic interface. -func (f *FunctionCase) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (f *CaseExpr) IsStatic() bool { if f.Value != nil && !f.Value.IsStatic() { return false } @@ -309,60 +234,15 @@ func (f *FunctionCase) IsStatic() bool { return true } -// castOperatopr is the operator type for cast function. -type castFunctionType int - -const ( - // CastFunction is CAST function. - CastFunction castFunctionType = iota + 1 - // ConvertFunction is CONVERT function. - ConvertFunction - // BinaryOperator is BINARY operator. - BinaryOperator -) - -// FunctionCast 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 FunctionCast struct { - txtNode - tpNode - // Expr is the expression to be converted. - Expr Expression - // Tp is the conversion type. - Tp *types.FieldType - // Cast, Convert and Binary share this struct. - FunctionType castFunctionType -} - -// IsStatic implements the Expression IsStatic interface. -func (f *FunctionCast) IsStatic() bool { - return f.Expr.IsStatic() -} - -// Accept implements Node Accept interface. -func (f *FunctionCast) 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.(Expression) - return v.Leave(f) -} - -// SubQuery represents a sub query. -type SubQuery struct { - txtNode - tpNode - nonStatic +// SubqueryExpr represents a sub query. +type SubqueryExpr struct { + exprNode // Query is the query SelectNode. - Query *SelectNode + Query *SelectStmt } // Accept implements Node Accept interface. -func (sq *SubQuery) Accept(v Visitor) (Node, bool) { +func (sq *SubqueryExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(sq) { return sq, false } @@ -370,30 +250,28 @@ func (sq *SubQuery) Accept(v Visitor) (Node, bool) { if !ok { return sq, false } - sq.Query = node.(*SelectNode) + sq.Query = node.(*SelectStmt) return v.Leave(sq) } -// CompareSubQuery is the expression for "expr cmp (select ...)". +// 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 CompareSubQuery struct { - txtNode - tpNode - nonStatic +type CompareSubqueryExpr struct { + exprNode // L is the left expression - L Expression + L ExprNode // Op is the comparison opcode. Op opcode.Op // R is the sub query for right expression. - R *SubQuery + R *SubqueryExpr // All is true, we should compare all records in subquery. All bool } // Accept implements Node Accept interface. -func (cs *CompareSubQuery) Accept(v Visitor) (Node, bool) { +func (cs *CompareSubqueryExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(cs) { return cs, false } @@ -401,91 +279,56 @@ func (cs *CompareSubQuery) Accept(v Visitor) (Node, bool) { if !ok { return cs, false } - cs.L = node.(Expression) + cs.L = node.(ExprNode) node, ok = cs.R.Accept(v) if !ok { return cs, false } - cs.R = node.(*SubQuery) + cs.R = node.(*SubqueryExpr) return v.Leave(cs) } -// ColumnRef represents a column reference. -type ColumnRef struct { - txtNode - tpNode - nonStatic +// 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 *ColumnRef) Accept(v Visitor) (Node, bool) { +func (cr *ColumnRefExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(cr) { return cr, false } return v.Leave(cr) } -// FunctionConvert 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 FunctionConvert struct { - txtNode - tpNode - // Expr is the expression to be converted. - Expr Expression - // Charset is the target character set to convert. - Charset string -} - -// IsStatic implements the Expression IsStatic interface. -func (f *FunctionConvert) IsStatic() bool { - return f.Expr.IsStatic() -} - -// Accept implements Node Accept interface. -func (f *FunctionConvert) 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.(Expression) - return v.Leave(f) -} - -// Default is the default expression using default value for a column. -type Default struct { - txtNode - tpNode - nonStatic +// 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 *Default) Accept(v Visitor) (Node, bool) { +func (d *DefaultExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(d) { return d, false } return v.Leave(d) } -// ExistsSubQuery is the expression for "exists (select ...)". +// ExistsSubqueryExpr is the expression for "exists (select ...)". // https://dev.mysql.com/doc/refman/5.7/en/exists-and-not-exists-subqueries.html -type ExistsSubQuery struct { - txtNode - tpNode - nonStatic +type ExistsSubqueryExpr struct { + exprNode // Sel is the sub query. - Sel *SubQuery + Sel *SubqueryExpr } // Accept implements Node Accept interface. -func (es *ExistsSubQuery) Accept(v Visitor) (Node, bool) { +func (es *ExistsSubqueryExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(es) { return es, false } @@ -493,55 +336,25 @@ func (es *ExistsSubQuery) Accept(v Visitor) (Node, bool) { if !ok { return es, false } - es.Sel = node.(*SubQuery) + es.Sel = node.(*SubqueryExpr) return v.Leave(es) } -// Extract is for time extract function. -// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_extract -type Extract struct { - txtNode - tpNode - - Unit string - Date Expression -} - -// Accept implements Node Accept interface. -func (ex *Extract) 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.(Expression) - return v.Leave(ex) -} - -// IsStatic implements the Expression IsStatic interface. -func (ex *Extract) IsStatic() bool { - return ex.Date.IsStatic() -} - -// PatternIn is the expression for in operator, like "expr in (1, 2, 3)" or "expr in (select c from t)". -type PatternIn struct { - txtNode - tpNode - nonStatic +// 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 Expression + Expr ExprNode // List is the list expression in compare list. - List []Expression + List []ExprNode // Not is true, the expression is "not in". Not bool // Sel is the sub query. - Sel *SubQuery + Sel *SubqueryExpr } // Accept implements Node Accept interface. -func (pi *PatternIn) Accept(v Visitor) (Node, bool) { +func (pi *PatternInExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(pi) { return pi, false } @@ -549,36 +362,35 @@ func (pi *PatternIn) Accept(v Visitor) (Node, bool) { if !ok { return pi, false } - pi.Expr = node.(Expression) + pi.Expr = node.(ExprNode) for i, val := range pi.List { node, ok = val.Accept(v) if !ok { return pi, false } - pi.List[i] = node.(Expression) + pi.List[i] = node.(ExprNode) } if pi.Sel != nil { node, ok = pi.Sel.Accept(v) if !ok { return pi, false } - pi.Sel = node.(*SubQuery) + pi.Sel = node.(*SubqueryExpr) } return v.Leave(pi) } -// IsNull is the expression for null check. -type IsNull struct { - txtNode - tpNode +// IsNullExpr is the expression for null check. +type IsNullExpr struct { + exprNode // Expr is the expression to be checked. - Expr Expression + Expr ExprNode // Not is true, the expression is "is not null". Not bool } // Accept implements Node Accept interface. -func (is *IsNull) Accept(v Visitor) (Node, bool) { +func (is *IsNullExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(is) { return is, false } @@ -586,21 +398,20 @@ func (is *IsNull) Accept(v Visitor) (Node, bool) { if !ok { return is, false } - is.Expr = node.(Expression) + is.Expr = node.(ExprNode) return v.Leave(is) } -// IsStatic implements the Expression IsStatic interface. -func (is *IsNull) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (is *IsNullExpr) IsStatic() bool { return is.Expr.IsStatic() } -// IsTruth is the expression for true/false check. -type IsTruth struct { - txtNode - tpNode +// IsTruthExpr is the expression for true/false check. +type IsTruthExpr struct { + exprNode // Expr is the expression to be checked. - Expr Expression + Expr ExprNode // Not is true, the expression is "is not true/false". Not bool // True indicates checking true or false. @@ -608,7 +419,7 @@ type IsTruth struct { } // Accept implements Node Accept interface. -func (is *IsTruth) Accept(v Visitor) (Node, bool) { +func (is *IsTruthExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(is) { return is, false } @@ -616,29 +427,28 @@ func (is *IsTruth) Accept(v Visitor) (Node, bool) { if !ok { return is, false } - is.Expr = node.(Expression) + is.Expr = node.(ExprNode) return v.Leave(is) } -// IsStatic implements the Expression IsStatic interface. -func (is *IsTruth) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (is *IsTruthExpr) IsStatic() bool { return is.Expr.IsStatic() } -// PatternLike is the expression for like operator, e.g, expr like "%123%" -type PatternLike struct { - txtNode - tpNode +// PatternLikeExpr is the expression for like operator, e.g, expr like "%123%" +type PatternLikeExpr struct { + exprNode // Expr is the expression to be checked. - Expr Expression + Expr ExprNode // Pattern is the like expression. - Pattern Expression + Pattern ExprNode // Not is true, the expression is "not like". Not bool } // Accept implements Node Accept interface. -func (pl *PatternLike) Accept(v Visitor) (Node, bool) { +func (pl *PatternLikeExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(pl) { return pl, false } @@ -646,112 +456,98 @@ func (pl *PatternLike) Accept(v Visitor) (Node, bool) { if !ok { return pl, false } - pl.Expr = node.(Expression) + pl.Expr = node.(ExprNode) node, ok = pl.Pattern.Accept(v) if !ok { return pl, false } - pl.Pattern = node.(Expression) + pl.Pattern = node.(ExprNode) return v.Leave(pl) } -// IsStatic implements the Expression IsStatic interface. -func (pl *PatternLike) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (pl *PatternLikeExpr) IsStatic() bool { return pl.Expr.IsStatic() && pl.Pattern.IsStatic() } -// ParamMarker expresion holds a place for another expression. +// ParamMarkerExpr expresion holds a place for another expression. // Used in parsing prepare statement. -type ParamMarker struct { - txtNode - tpNode - // Expr is the expression to be evaluated in this place holder. - Expr Expression +type ParamMarkerExpr struct { + exprNode } // Accept implements Node Accept interface. -func (pm *ParamMarker) Accept(v Visitor) (Node, bool) { +func (pm *ParamMarkerExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(pm) { return pm, false } - node, ok := pm.Expr.Accept(v) - if !ok { - return pm, false - } - pm.Expr = node.(Expression) return v.Leave(pm) } -// IsStatic implements the Expression IsStatic interface. -func (pm *ParamMarker) IsStatic() bool { - return pm.Expr.IsStatic() -} - -// PExpr is the parenthesis expression. -type PExpr struct { - txtNode - tpNode - // Expr is the expression in parenthesis. - Expr Expression +// ParenthesesExpr is the parentheses expression. +type ParenthesesExpr struct { + exprNode + // Expr is the expression in parentheses. + Expr ExprNode } // Accept implements Node Accept interface. -func (p *PExpr) Accept(v Visitor) (Node, bool) { +func (p *ParenthesesExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(p) { return p, false } - node, ok := p.Expr.Accept(v) - if !ok { - return p, false + if p.Expr != nil { + node, ok := p.Expr.Accept(v) + if !ok { + return p, false + } + p.Expr = node.(ExprNode) } - p.Expr = node.(Expression) return v.Leave(p) } -// IsStatic implements the Expression IsStatic interface. -func (p *PExpr) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (p *ParenthesesExpr) IsStatic() bool { return p.Expr.IsStatic() } -// Position is the expression for order by and group by position. +// 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 Position struct { - txtNode - tpNode +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 Expression IsStatic interface. -func (p *Position) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (p *PositionExpr) IsStatic() bool { return true } // Accept implements Node Accept interface. -func (p *Position) Accept(v Visitor) (Node, bool) { +func (p *PositionExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(p) { return p, false } return v.Leave(p) } -// PatternRegexp is the pattern expression for pattern match. -type PatternRegexp struct { - txtNode - tpNode +// PatternRegexpExpr is the pattern expression for pattern match. +type PatternRegexpExpr struct { + exprNode // Expr is the expression to be checked. - Expr Expression + Expr ExprNode // Pattern is the expression for pattern. - Pattern Expression + Pattern ExprNode // Not is true, the expression is "not rlike", Not bool } // Accept implements Node Accept interface. -func (p *PatternRegexp) Accept(v Visitor) (Node, bool) { +func (p *PatternRegexpExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(p) { return p, false } @@ -759,31 +555,30 @@ func (p *PatternRegexp) Accept(v Visitor) (Node, bool) { if !ok { return p, false } - p.Expr = node.(Expression) + p.Expr = node.(ExprNode) node, ok = p.Pattern.Accept(v) if !ok { return p, false } - p.Pattern = node.(Expression) + p.Pattern = node.(ExprNode) return v.Leave(p) } -// IsStatic implements the Expression IsStatic interface. -func (p *PatternRegexp) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (p *PatternRegexpExpr) IsStatic() bool { return p.Expr.IsStatic() && p.Pattern.IsStatic() } -// Row is the expression for row constructor. +// RowExpr is the expression for row constructor. // See https://dev.mysql.com/doc/refman/5.7/en/row-subqueries.html -type Row struct { - txtNode - tpNode +type RowExpr struct { + exprNode - Values []Expression + Values []ExprNode } // Accept implements Node Accept interface. -func (r *Row) Accept(v Visitor) (Node, bool) { +func (r *RowExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(r) { return r, false } @@ -792,13 +587,13 @@ func (r *Row) Accept(v Visitor) (Node, bool) { if !ok { return r, false } - r.Values[i] = node.(Expression) + r.Values[i] = node.(ExprNode) } return v.Leave(r) } -// IsStatic implements the Expression IsStatic interface. -func (r *Row) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (r *RowExpr) IsStatic() bool { for _, v := range r.Values { if !v.IsStatic() { return false @@ -807,102 +602,17 @@ func (r *Row) IsStatic() bool { return true } -// FunctionSubstring returns the substring as specified. -// See: https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_substring -type FunctionSubstring struct { - txtNode - tpNode - - StrExpr Expression - Pos Expression - Len Expression -} - -// Accept implements Node Accept interface. -func (fs *FunctionSubstring) Accept(v Visitor) (Node, bool) { - if !v.Enter(fs) { - return fs, false - } - node, ok := fs.StrExpr.Accept(v) - if !ok { - return fs, false - } - fs.StrExpr = node.(Expression) - node, ok = fs.Pos.Accept(v) - if !ok { - return fs, false - } - fs.Pos = node.(Expression) - node, ok = fs.Len.Accept(v) - if !ok { - return fs, false - } - fs.Len = node.(Expression) - return v.Leave(fs) -} - -// IsStatic implements the Expression IsStatic interface. -func (fs *FunctionSubstring) IsStatic() bool { - return fs.StrExpr.IsStatic() && fs.Pos.IsStatic() && fs.Len.IsStatic() -} - -const ( - // TrimBothDefault trims from both direction by default. - TrimBothDefault = iota - // TrimBoth trims from both direction with explicit notation. - TrimBoth - // TrimLeading trims from left. - TrimLeading - // TrimTrailing trims from right. - TrimTrailing -) - -// FunctionTrim remove leading/trailing/both remstr. -// See: https://dev.mysql.com/doc/refman/5.7/en/string-functions.html#function_trim -type FunctionTrim struct { - txtNode - tpNode - - Str Expression - RemStr Expression - Direction int -} - -// Accept implements Node Accept interface. -func (ft *FunctionTrim) Accept(v Visitor) (Node, bool) { - if !v.Enter(ft) { - return ft, false - } - node, ok := ft.Str.Accept(v) - if !ok { - return ft, false - } - ft.Str = node.(Expression) - node, ok = ft.RemStr.Accept(v) - if !ok { - return ft, false - } - ft.RemStr = node.(Expression) - return v.Leave(ft) -} - -// IsStatic implements the Expression IsStatic interface. -func (ft *FunctionTrim) IsStatic() bool { - return ft.Str.IsStatic() && ft.RemStr.IsStatic() -} - -// UnaryOperation is the expression for unary operator. -type UnaryOperation struct { - txtNode - tpNode +// 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 Expression + V ExprNode } // Accept implements Node Accept interface. -func (u *UnaryOperation) Accept(v Visitor) (Node, bool) { +func (u *UnaryOperationExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(u) { return u, false } @@ -910,26 +620,24 @@ func (u *UnaryOperation) Accept(v Visitor) (Node, bool) { if !ok { return u, false } - u.V = node.(Expression) + u.V = node.(ExprNode) return v.Leave(u) } -// IsStatic implements the Expression IsStatic interface. -func (u *UnaryOperation) IsStatic() bool { +// IsStatic implements the ExprNode IsStatic interface. +func (u *UnaryOperationExpr) IsStatic() bool { return u.V.IsStatic() } -// Values is the expression used in INSERT VALUES -type Values struct { - txtNode - tpNode - nonStatic +// ValuesExpr is the expression used in INSERT VALUES +type ValuesExpr struct { + exprNode // model.CIStr is column name. - Column *ColumnRef + Column *ColumnRefExpr } // Accept implements Node Accept interface. -func (va *Values) Accept(v Visitor) (Node, bool) { +func (va *ValuesExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(va) { return va, false } @@ -937,15 +645,13 @@ func (va *Values) Accept(v Visitor) (Node, bool) { if !ok { return va, false } - va.Column = node.(*ColumnRef) + va.Column = node.(*ColumnRefExpr) return v.Leave(va) } -// Variable is the expression for variable. -type Variable struct { - txtNode - tpNode - nonStatic +// VariableExpr is the expression for variable. +type VariableExpr struct { + exprNode // Name is the variable name. Name string // IsGlobal indicates whether this variable is global. @@ -955,7 +661,7 @@ type Variable struct { } // Accept implements Node Accept interface. -func (va *Variable) Accept(v Visitor) (Node, bool) { +func (va *VariableExpr) Accept(v Visitor) (Node, bool) { if !v.Enter(va) { return va, false } 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 index a6d29a3b06..1e6d0feade 100644 --- a/ast/misc.go +++ b/ast/misc.go @@ -1,15 +1,31 @@ +// 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 ( - _ Node = &ExplainStmt{} - _ Node = &PreparedStmt{} - _ Node = &DeallocateStmt{} - _ Node = &ExecuteStmt{} - _ Node = &BeginStmt{} - _ Node = &CommitStmt{} - _ Node = &RollbackStmt{} + _ StmtNode = &ExplainStmt{} + _ StmtNode = &PrepareStmt{} + _ StmtNode = &DeallocateStmt{} + _ StmtNode = &ExecuteStmt{} + _ StmtNode = &ShowStmt{} + _ StmtNode = &BeginStmt{} + _ StmtNode = &CommitStmt{} + _ StmtNode = &RollbackStmt{} + _ StmtNode = &UseStmt{} + _ StmtNode = &SetStmt{} + _ Node = &VariableAssignment{} - _ Node = &SetStmt{} ) // AuthOption is used for parsing create use statement. @@ -25,93 +41,208 @@ type AuthOption struct { // or get columns information in a table. // See: https://dev.mysql.com/doc/refman/5.7/en/explain.html type ExplainStmt struct { - txtNode + stmtNode - S Node + Stmt DMLNode } -// PreparedStmt is a statement to prepares a SQL statement which contains placeholders, +// 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 PreparedStmt struct { - txtNode +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 - SQLVar *Variable - SQLStmt Node // The parsed statement from sql text with placeholder - Params []*ParamMarker + 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 { - txtNode + 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 { - txtNode + stmtNode Name string ID uint32 // For binary protocol, there is no Name but only ID - UsingVars []Expression + 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 { - txtNode + stmtNode - Target int // Databases/Tables/Columns/.... - DBName string - Table *TableRef // Used for showing columns. - ColumnName string // Used for `desc table column`. - Flag int // Some flag parsed from sql, such as FULL. - Full bool + 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 *PatternLike - Where Expression + 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 { - txtNode + 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 { - txtNode + 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 { - txtNode + 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 { - txtNode + 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 { - txtNode + node Name string - Value Expression + Value ExprNode IsGlobal bool IsSystem bool } @@ -125,13 +256,13 @@ func (va *VariableAssignment) Accept(v Visitor) (Node, bool) { if !ok { return va, false } - va.Value = node.(Expression) + va.Value = node.(ExprNode) return v.Leave(va) } // SetStmt is the statement to set variables. type SetStmt struct { - txtNode + stmtNode // Variables is the list of variable assignment. Variables []*VariableAssignment } From 2ddd70898d3a6f4e100e0fb8961ba3186ba75781 Mon Sep 17 00:00:00 2001 From: siddontang Date: Wed, 14 Oct 2015 21:39:10 +0800 Subject: [PATCH 27/58] plans: return mysql error to pass sqlalchemy test --- plan/plans/show.go | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/plan/plans/show.go b/plan/plans/show.go index 0dac38c217..87eff2e91a 100644 --- a/plan/plans/show.go +++ b/plan/plans/show.go @@ -194,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 } From 7716e6024d3974e6211a06f43d7ba1c02cebdc22 Mon Sep 17 00:00:00 2001 From: siddontang Date: Thu, 15 Oct 2015 09:48:41 +0800 Subject: [PATCH 28/58] plans: must use a new index plan for filter. --- plan/plans/index.go | 24 ++++++++++++++++-------- 1 file changed, 16 insertions(+), 8 deletions(-) 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) From 6350100d61da5689b18c6536c86917aebf6a6aad Mon Sep 17 00:00:00 2001 From: siddontang Date: Thu, 15 Oct 2015 09:49:55 +0800 Subject: [PATCH 29/58] rsets: should check where condition even using index plan. --- rset/rsets/where.go | 28 +++++++++++++++++++++++----- 1 file changed, 23 insertions(+), 5 deletions(-) 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 } From c3ca28f32c71d8e6ba175cf1304fe0297ed36b73 Mon Sep 17 00:00:00 2001 From: siddontang Date: Thu, 15 Oct 2015 09:51:29 +0800 Subject: [PATCH 30/58] tidb: add test. --- tidb_test.go | 36 ++++++++++++++++++++++++++++++++++++ 1 file changed, 36 insertions(+) diff --git a/tidb_test.go b/tidb_test.go index 84bdc17462..20ae252e28 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) { @@ -848,6 +863,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) { @@ -1151,3 +1180,10 @@ 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]...) + } +} From a49849a93dd95b682d97a34c3e24a3944de00d28 Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Thu, 15 Oct 2015 14:48:59 +0800 Subject: [PATCH 31/58] ast: address comments. --- ast/ast.go | 2 +- ast/base.go | 6 +++--- 2 files changed, 4 insertions(+), 4 deletions(-) diff --git a/ast/ast.go b/ast/ast.go index a857aa72de..bcf0db2db1 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -41,7 +41,7 @@ type ExprNode interface { IsStatic() bool // SetType sets type to the expression. SetType(tp *types.FieldType) - // GetType gets type of the expression. + // GetType returns type of the expression. GetType() *types.FieldType } diff --git a/ast/base.go b/ast/base.go index 17d63d164e..5767cd9ebe 100644 --- a/ast/base.go +++ b/ast/base.go @@ -18,17 +18,17 @@ 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 { - txt string + text string } // SetText implements Node interface. func (n *node) SetText(text string) { - n.txt = text + n.text = text } // Text implements Node interface. func (n *node) Text() string { - return n.txt + return n.text } // stmtNode implements StmtNode interface. From 41465ddfa8aa4df542ba94952240512c0a956d74 Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Thu, 15 Oct 2015 15:16:17 +0800 Subject: [PATCH 32/58] ast: improve comment. --- ast/ast.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ast/ast.go b/ast/ast.go index bcf0db2db1..2cec3ed90c 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -41,7 +41,7 @@ type ExprNode interface { IsStatic() bool // SetType sets type to the expression. SetType(tp *types.FieldType) - // GetType returns type of the expression. + // GetType gets the evaluation type of the expression. GetType() *types.FieldType } From bb43093b7c34ecc54bc6da71a01be94354085baf Mon Sep 17 00:00:00 2001 From: Ewan Chou Date: Thu, 15 Oct 2015 15:17:38 +0800 Subject: [PATCH 33/58] ast: improve comment. --- ast/ast.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/ast/ast.go b/ast/ast.go index 2cec3ed90c..5caa14a8ae 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -39,7 +39,7 @@ type ExprNode interface { Node // IsStatic means it can be evaluated independently. IsStatic() bool - // SetType sets type to the expression. + // SetType sets evaluation type to the expression. SetType(tp *types.FieldType) // GetType gets the evaluation type of the expression. GetType() *types.FieldType From a4aa3e72e6b8622fab608f8eb9c238aa0dcf3a07 Mon Sep 17 00:00:00 2001 From: wink Date: Thu, 15 Oct 2015 15:49:29 +0800 Subject: [PATCH 34/58] parser/parser.y: add StringType OptCharset OptCollate support to support sql like "Create Table test(a CHAR(10) CHARACTER SET utf8)", see details: "https://dev.mysql.com/doc/refman/5.0/en/create-table.html", data_type: ... | CHAR[(length)] [BINARY] [CHARACTER SET charset_name] [COLLATE collation_name] --- parser/parser.y | 8 ++++++-- parser/parser_test.go | 5 +++++ 2 files changed, 11 insertions(+), 2 deletions(-) diff --git a/parser/parser.y b/parser/parser.y index 0f9801ffca..a9b7059067 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -3861,21 +3861,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..28fae5e330 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -442,6 +442,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 From 22143ecaf1569b537c2c6205f2347a0715cec15f Mon Sep 17 00:00:00 2001 From: ngaut Date: Thu, 15 Oct 2015 17:03:39 +0800 Subject: [PATCH 35/58] meta: Disable generating global ID concurrently. --- meta/meta.go | 9 ++++++++- meta/meta_test.go | 4 ++-- 2 files changed, 10 insertions(+), 3 deletions(-) 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) From 207ba2f1f332a9ea96d7fcb5bdbdc023d0ccf203 Mon Sep 17 00:00:00 2001 From: shenli Date: Thu, 15 Oct 2015 17:17:04 +0800 Subject: [PATCH 36/58] *: Add sql helper to make operations on meta table easier. --- session.go | 29 +++ stmt/stmts/account_manage.go | 83 ++---- "util/sqlhelper/\\" | 477 +++++++++++++++++++++++++++++++++++ util/sqlhelper/sql_helper.go | 20 ++ 4 files changed, 545 insertions(+), 64 deletions(-) create mode 100644 "util/sqlhelper/\\" create mode 100644 util/sqlhelper/sql_helper.go diff --git a/session.go b/session.go index 5073e97bde..d2dbe25883 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/sqlhelper" ) // Session context @@ -228,6 +230,33 @@ func (s *session) Retry() error { return nil } +// ExecRestrictedSQL implements SQLHelper interface. +// This is used for execute some restricted sql statement. +func (s *session) ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recordset, error) { + if ctx.Value(&sqlhelper.KeyType{}) != nil { + return nil, nil + } + statements, err := Compile(sql) + if err != nil { + log.Errorf("Syntax error: %s", sql) + return nil, errors.Trace(err) + } + if len(statements) == 0 { + log.Warnf("No statement to exec for %s", sql) + } + st := statements[0] + // Check statement for some restriction + // For example only support DML on system meta table. + log.Infof("Executing %s [%s]", st, sql) + ctx.SetValue(&sqlhelper.KeyType{}, true) + defer ctx.ClearValue(&sqlhelper.KeyType{}) + rs, err := st.Exec(ctx) + if err != nil { + return nil, errors.Trace(err) + } + return rs, err +} + func (s *session) Execute(sql string) ([]rset.Recordset, error) { statements, err := Compile(sql) if err != nil { diff --git a/stmt/stmts/account_manage.go b/stmt/stmts/account_manage.go index 072673b1e8..5bed6510cf 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/sqlhelper" ) /************************************************************************************ @@ -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.(sqlhelper.SQLHelper).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.(sqlhelper.SQLHelper).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.(sqlhelper.SQLHelper).ExecRestrictedSQL(ctx, sql) + return nil, errors.Trace(err) } diff --git "a/util/sqlhelper/\\" "b/util/sqlhelper/\\" new file mode 100644 index 0000000000..9a0431b706 --- /dev/null +++ "b/util/sqlhelper/\\" @@ -0,0 +1,477 @@ +// Copyright 2013 The ql Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSES/QL-LICENSE file. + +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package tidb + +import ( + "bytes" + "encoding/json" + "fmt" + "strings" + "sync" + "sync/atomic" + "time" + + "github.com/juju/errors" + "github.com/ngaut/log" + "github.com/pingcap/tidb/field" + "github.com/pingcap/tidb/kv" + mysql "github.com/pingcap/tidb/mysqldef" + "github.com/pingcap/tidb/rset" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/sessionctx/db" + "github.com/pingcap/tidb/sessionctx/forupdate" + "github.com/pingcap/tidb/sessionctx/variable" + "github.com/pingcap/tidb/stmt" + "github.com/pingcap/tidb/stmt/stmts" + "github.com/pingcap/tidb/util" + "github.com/pingcap/tidb/sqlhelper" + "github.com/pingcap/tidb/util/errors2" +) + +// Session context +type Session interface { + Status() uint16 // Flag of current status, such as autocommit + LastInsertID() uint64 // Last inserted auto_increment id + AffectedRows() uint64 // Affected rows by lastest executed stmt + Execute(sql string) ([]rset.Recordset, error) // Execute a sql statement + String() string // For debug + FinishTxn(rollback bool) error + // For execute prepare statement in binary protocol + PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*field.ResultField, err error) + // Execute a prepared statement + ExecutePreparedStmt(stmtID uint32, param ...interface{}) (rset.Recordset, error) + DropPreparedStmt(stmtID uint32) error + SetClientCapability(uint32) // Set client capability flags + Close() error + Retry() error + Auth(user string, auth []byte, salt []byte) bool +} + +var ( + _ Session = (*session)(nil) + sessionID int64 + sessionMu sync.Mutex +) + +type stmtRecord struct { + stmtID uint32 + st stmt.Statement + params []interface{} +} + +type stmtHistory struct { + history []*stmtRecord +} + +func (h *stmtHistory) add(stmtID uint32, st stmt.Statement, params ...interface{}) { + s := &stmtRecord{ + stmtID: stmtID, + st: st, + params: append(([]interface{})(nil), params...), + } + + h.history = append(h.history, s) +} + +func (h *stmtHistory) reset() { + if len(h.history) > 0 { + h.history = h.history[:0] + } +} + +func (h *stmtHistory) clone() *stmtHistory { + nh := *h + nh.history = make([]*stmtRecord, len(h.history)) + copy(nh.history, h.history) + return &nh +} + +type session struct { + txn kv.Transaction // Current transaction + args []interface{} // Statment execution args, this should be cleaned up after exec + values map[fmt.Stringer]interface{} + store kv.Storage + sid int64 + history stmtHistory +} + +func (s *session) Status() uint16 { + return variable.GetSessionVars(s).Status +} + +func (s *session) LastInsertID() uint64 { + return variable.GetSessionVars(s).LastInsertID +} + +func (s *session) AffectedRows() uint64 { + return variable.GetSessionVars(s).AffectedRows +} + +func (s *session) resetHistory() { + s.ClearValue(forupdate.ForUpdateKey) + s.history.reset() +} + +func (s *session) SetClientCapability(capability uint32) { + variable.GetSessionVars(s).ClientCapability = capability +} + +func (s *session) FinishTxn(rollback bool) error { + // transaction has already been committed or rolled back + if s.txn == nil { + return nil + } + defer func() { + s.txn = nil + variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusInTrans, false) + }() + + if rollback { + return s.txn.Rollback() + } + + err := s.txn.Commit() + if err != nil { + log.Warnf("txn:%s, %v", s.txn, err) + return errors.Trace(err) + } + + s.resetHistory() + return nil +} + +func (s *session) String() string { + // TODO: how to print binded context in values appropriately? + data := map[string]interface{}{ + "currDBName": db.GetCurrentSchema(s), + "sid": s.sid, + } + + if s.txn != nil { + // if txn is committed or rolled back, txn is nil. + data["txn"] = s.txn.String() + } + + b, _ := json.MarshalIndent(data, "", " ") + return string(b) +} + +func needRetry(st stmt.Statement) bool { + switch st.(type) { + case *stmts.PreparedStmt, *stmts.ShowStmt, *stmts.DoStmt: + return false + default: + return true + } +} + +func isPreparedStmt(st stmt.Statement) bool { + switch st.(type) { + case *stmts.PreparedStmt: + return true + default: + return false + } +} + +func (s *session) Retry() error { + nh := s.history.clone() + defer func() { + s.history.history = nh.history + }() + + if forUpdate := s.Value(forupdate.ForUpdateKey); forUpdate != nil { + return errors.Errorf("can not retry select for update statement") + } + + var err error + for { + s.resetHistory() + s.FinishTxn(true) + success := true + for _, sr := range nh.history { + st := sr.st + // Skip prepare statement + if !needRetry(st) { + continue + } + log.Warnf("Retry %s", st.OriginText()) + _, err = runStmt(s, st) + if err != nil { + if errors2.ErrorEqual(err, kv.ErrConditionNotMatch) { + success = false + break + } + log.Warnf("session:%v, err:%v", s, err) + return errors.Trace(err) + } + } + if success { + break + } + } + + return nil +} + +// ExecRestrictedSQL implements. +// This is used for execute some restricted sql statement. +func (s *session) ExecRestrictedSQL(sql string) ([]rset.Recordset, error) { + if ctx.Value(&sql.MetaQueryType{}) != nil { + return nil, nil + } + statements, err := Compile(sql) + if err != nil { + log.Errorf("Syntax error: %s", sql) + return nil, errors.Trace(err) + } + if len(statements) == 0 { + log.Warnf("Now statement to exec for %s", sql) + } + st := statements[0] + // Check statement for some restriction + // For example only support DML on system meta table. + log.Infof("Executing %s [%s]", st, sql) + ctx.SetValue(&sql.MetaQueryType{}, true) + defer ctx.ClearValue(&sql.MetaQueryType{}) + return st.Exec(ctx) +} + +func (s *session) Execute(sql string) ([]rset.Recordset, error) { + statements, err := Compile(sql) + if err != nil { + log.Errorf("Syntax error: %s", sql) + log.Errorf("Error occurs at %s.", err) + return nil, errors.Trace(err) + } + + var rs []rset.Recordset + + for _, st := range statements { + r, err := runStmt(s, st) + if err != nil { + log.Warnf("session:%v, err:%v", s, err) + return nil, errors.Trace(err) + } + + // Record executed query + if isPreparedStmt(st) { + ps := st.(*stmts.PreparedStmt) + s.history.add(ps.ID, st) + } else { + s.history.add(0, st) + } + + if r != nil { + rs = append(rs, r) + } + } + + return rs, nil +} + +// For execute prepare statement in binary protocol +func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*field.ResultField, err error) { + return prepareStmt(s, sql) +} + +// checkArgs makes sure all the arguments' types are known and can be handled. +// integer types are converted to int64 and uint64, time.Time is converted to mysql.Time. +// time.Duration is converted to mysql.Duration, other known types are leaved as it is. +func checkArgs(args ...interface{}) error { + for i, v := range args { + switch x := v.(type) { + case bool: + if x { + args[i] = int64(1) + } else { + args[i] = int64(0) + } + case int8: + args[i] = int64(x) + case int16: + args[i] = int64(x) + case int32: + args[i] = int64(x) + case int: + args[i] = int64(x) + case uint8: + args[i] = uint64(x) + case uint16: + args[i] = uint64(x) + case uint32: + args[i] = uint64(x) + case uint: + args[i] = uint64(x) + case int64: + case uint64: + case float32: + case float64: + case string: + case []byte: + case time.Duration: + args[i] = mysql.Duration{Duration: x} + case time.Time: + args[i] = mysql.Time{Time: x, Type: mysql.TypeDatetime} + case nil: + default: + return errors.Errorf("cannot use arg[%d] (type %T):unsupported type", i, v) + } + } + return nil +} + +// Execute a prepared statement +func (s *session) ExecutePreparedStmt(stmtID uint32, args ...interface{}) (rset.Recordset, error) { + err := checkArgs(args...) + if err != nil { + return nil, err + } + + st := &stmts.ExecuteStmt{ID: stmtID} + s.history.add(stmtID, st, args...) + return runStmt(s, st, args...) +} + +func (s *session) DropPreparedStmt(stmtID uint32) error { + return dropPreparedStmt(s, stmtID) +} + +// If forceNew is true, GetTxn() must return a new transaction. +// In this situation, if current transaction is still in progress, +// there will be an implicit commit and create a new transaction. +func (s *session) GetTxn(forceNew bool) (kv.Transaction, error) { + var err error + if s.txn == nil { + s.resetHistory() + s.txn, err = s.store.Begin() + if err != nil { + return nil, err + } + if !variable.IsAutocommit(s) { + variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusInTrans, true) + } + log.Infof("New txn:%s in session:%d", s.txn, s.sid) + return s.txn, nil + } + if forceNew { + err = s.txn.Commit() + variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusInTrans, false) + if err != nil { + return nil, err + } + s.resetHistory() + s.txn, err = s.store.Begin() + if err != nil { + return nil, err + } + if !variable.IsAutocommit(s) { + variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusInTrans, true) + } + log.Warnf("Force new txn:%s in session:%d", s.txn, s.sid) + } + return s.txn, nil +} + +func (s *session) SetValue(key fmt.Stringer, value interface{}) { + s.values[key] = value +} + +func (s *session) Value(key fmt.Stringer) interface{} { + value := s.values[key] + return value +} + +func (s *session) ClearValue(key fmt.Stringer) { + delete(s.values, key) +} + +// Close function does some clean work when session end. +func (s *session) Close() error { + return s.FinishTxn(true) +} + +func (s *session) Auth(user string, auth []byte, salt []byte) bool { + strs := strings.Split(user, "@") + if len(strs) != 2 { + log.Warnf("Invalid format for user: %s", user) + return false + } + // Get user password. + name := strs[0] + host := strs[1] + authSQL := fmt.Sprintf("SELECT Password FROM %s.%s WHERE User=\"%s\" and Host=\"%s\";", mysql.SystemDB, mysql.UserTable, name, host) + rs, err := s.Execute(authSQL) + if err != nil { + log.Warnf("Encounter error when auth user %s. Error: %v", user, err) + return false + } + if len(rs) == 0 { + return false + } + row, err := rs[0].Next() + if err != nil { + log.Warnf("Encounter error when auth user %s. Error: %v", user, err) + return false + } + if row == nil || len(row.Data) == 0 { + return false + } + pwd, ok := row.Data[0].(string) + if !ok { + return false + } + hpwd, err := util.DecodePassword(pwd) + if err != nil { + log.Errorf("Decode password string error %v", err) + return false + } + checkAuth := util.CalcPassword(salt, hpwd) + if !bytes.Equal(auth, checkAuth) { + return false + } + variable.GetSessionVars(s).SetCurrentUser(user) + return true +} + +// CreateSession creates a new session environment. +func CreateSession(store kv.Storage) (Session, error) { + s := &session{ + values: make(map[fmt.Stringer]interface{}), + store: store, + sid: atomic.AddInt64(&sessionID, 1), + } + domain, err := domap.Get(store) + if err != nil { + return nil, err + } + sessionctx.BindDomain(s, domain) + + variable.BindSessionVars(s) + variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusAutocommit, true) + sessionMu.Lock() + defer sessionMu.Unlock() + + _, ok := storeBootstrapped[store.UUID()] + if !ok { + bootstrap(s) + storeBootstrapped[store.UUID()] = true + } + // Add auth here + return s, nil +} diff --git a/util/sqlhelper/sql_helper.go b/util/sqlhelper/sql_helper.go new file mode 100644 index 0000000000..8960a7a768 --- /dev/null +++ b/util/sqlhelper/sql_helper.go @@ -0,0 +1,20 @@ +package sqlhelper + +import ( + "github.com/pingcap/tidb/context" + "github.com/pingcap/tidb/rset" +) + +// KeyType is a dummy type to avoid naming collision in session. +type KeyType struct{} + +// String implements Stringer interface. +func (k *KeyType) String() string { + return "sql_helper" +} + +// SQLHelper is an interface provides executing restricted sql statement. +type SQLHelper interface { + // ExecRestrictedSQL run sql statement in ctx with some restriction. + ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recordset, error) +} From 2bf61835db6b9c3117862415d3ec9a19bed22c77 Mon Sep 17 00:00:00 2001 From: shenli Date: Thu, 15 Oct 2015 17:19:51 +0800 Subject: [PATCH 37/58] sqlhelper: Remove useless file --- "util/sqlhelper/\\" | 477 -------------------------------------------- 1 file changed, 477 deletions(-) delete mode 100644 "util/sqlhelper/\\" diff --git "a/util/sqlhelper/\\" "b/util/sqlhelper/\\" deleted file mode 100644 index 9a0431b706..0000000000 --- "a/util/sqlhelper/\\" +++ /dev/null @@ -1,477 +0,0 @@ -// Copyright 2013 The ql Authors. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSES/QL-LICENSE file. - -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package tidb - -import ( - "bytes" - "encoding/json" - "fmt" - "strings" - "sync" - "sync/atomic" - "time" - - "github.com/juju/errors" - "github.com/ngaut/log" - "github.com/pingcap/tidb/field" - "github.com/pingcap/tidb/kv" - mysql "github.com/pingcap/tidb/mysqldef" - "github.com/pingcap/tidb/rset" - "github.com/pingcap/tidb/sessionctx" - "github.com/pingcap/tidb/sessionctx/db" - "github.com/pingcap/tidb/sessionctx/forupdate" - "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/stmt" - "github.com/pingcap/tidb/stmt/stmts" - "github.com/pingcap/tidb/util" - "github.com/pingcap/tidb/sqlhelper" - "github.com/pingcap/tidb/util/errors2" -) - -// Session context -type Session interface { - Status() uint16 // Flag of current status, such as autocommit - LastInsertID() uint64 // Last inserted auto_increment id - AffectedRows() uint64 // Affected rows by lastest executed stmt - Execute(sql string) ([]rset.Recordset, error) // Execute a sql statement - String() string // For debug - FinishTxn(rollback bool) error - // For execute prepare statement in binary protocol - PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*field.ResultField, err error) - // Execute a prepared statement - ExecutePreparedStmt(stmtID uint32, param ...interface{}) (rset.Recordset, error) - DropPreparedStmt(stmtID uint32) error - SetClientCapability(uint32) // Set client capability flags - Close() error - Retry() error - Auth(user string, auth []byte, salt []byte) bool -} - -var ( - _ Session = (*session)(nil) - sessionID int64 - sessionMu sync.Mutex -) - -type stmtRecord struct { - stmtID uint32 - st stmt.Statement - params []interface{} -} - -type stmtHistory struct { - history []*stmtRecord -} - -func (h *stmtHistory) add(stmtID uint32, st stmt.Statement, params ...interface{}) { - s := &stmtRecord{ - stmtID: stmtID, - st: st, - params: append(([]interface{})(nil), params...), - } - - h.history = append(h.history, s) -} - -func (h *stmtHistory) reset() { - if len(h.history) > 0 { - h.history = h.history[:0] - } -} - -func (h *stmtHistory) clone() *stmtHistory { - nh := *h - nh.history = make([]*stmtRecord, len(h.history)) - copy(nh.history, h.history) - return &nh -} - -type session struct { - txn kv.Transaction // Current transaction - args []interface{} // Statment execution args, this should be cleaned up after exec - values map[fmt.Stringer]interface{} - store kv.Storage - sid int64 - history stmtHistory -} - -func (s *session) Status() uint16 { - return variable.GetSessionVars(s).Status -} - -func (s *session) LastInsertID() uint64 { - return variable.GetSessionVars(s).LastInsertID -} - -func (s *session) AffectedRows() uint64 { - return variable.GetSessionVars(s).AffectedRows -} - -func (s *session) resetHistory() { - s.ClearValue(forupdate.ForUpdateKey) - s.history.reset() -} - -func (s *session) SetClientCapability(capability uint32) { - variable.GetSessionVars(s).ClientCapability = capability -} - -func (s *session) FinishTxn(rollback bool) error { - // transaction has already been committed or rolled back - if s.txn == nil { - return nil - } - defer func() { - s.txn = nil - variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusInTrans, false) - }() - - if rollback { - return s.txn.Rollback() - } - - err := s.txn.Commit() - if err != nil { - log.Warnf("txn:%s, %v", s.txn, err) - return errors.Trace(err) - } - - s.resetHistory() - return nil -} - -func (s *session) String() string { - // TODO: how to print binded context in values appropriately? - data := map[string]interface{}{ - "currDBName": db.GetCurrentSchema(s), - "sid": s.sid, - } - - if s.txn != nil { - // if txn is committed or rolled back, txn is nil. - data["txn"] = s.txn.String() - } - - b, _ := json.MarshalIndent(data, "", " ") - return string(b) -} - -func needRetry(st stmt.Statement) bool { - switch st.(type) { - case *stmts.PreparedStmt, *stmts.ShowStmt, *stmts.DoStmt: - return false - default: - return true - } -} - -func isPreparedStmt(st stmt.Statement) bool { - switch st.(type) { - case *stmts.PreparedStmt: - return true - default: - return false - } -} - -func (s *session) Retry() error { - nh := s.history.clone() - defer func() { - s.history.history = nh.history - }() - - if forUpdate := s.Value(forupdate.ForUpdateKey); forUpdate != nil { - return errors.Errorf("can not retry select for update statement") - } - - var err error - for { - s.resetHistory() - s.FinishTxn(true) - success := true - for _, sr := range nh.history { - st := sr.st - // Skip prepare statement - if !needRetry(st) { - continue - } - log.Warnf("Retry %s", st.OriginText()) - _, err = runStmt(s, st) - if err != nil { - if errors2.ErrorEqual(err, kv.ErrConditionNotMatch) { - success = false - break - } - log.Warnf("session:%v, err:%v", s, err) - return errors.Trace(err) - } - } - if success { - break - } - } - - return nil -} - -// ExecRestrictedSQL implements. -// This is used for execute some restricted sql statement. -func (s *session) ExecRestrictedSQL(sql string) ([]rset.Recordset, error) { - if ctx.Value(&sql.MetaQueryType{}) != nil { - return nil, nil - } - statements, err := Compile(sql) - if err != nil { - log.Errorf("Syntax error: %s", sql) - return nil, errors.Trace(err) - } - if len(statements) == 0 { - log.Warnf("Now statement to exec for %s", sql) - } - st := statements[0] - // Check statement for some restriction - // For example only support DML on system meta table. - log.Infof("Executing %s [%s]", st, sql) - ctx.SetValue(&sql.MetaQueryType{}, true) - defer ctx.ClearValue(&sql.MetaQueryType{}) - return st.Exec(ctx) -} - -func (s *session) Execute(sql string) ([]rset.Recordset, error) { - statements, err := Compile(sql) - if err != nil { - log.Errorf("Syntax error: %s", sql) - log.Errorf("Error occurs at %s.", err) - return nil, errors.Trace(err) - } - - var rs []rset.Recordset - - for _, st := range statements { - r, err := runStmt(s, st) - if err != nil { - log.Warnf("session:%v, err:%v", s, err) - return nil, errors.Trace(err) - } - - // Record executed query - if isPreparedStmt(st) { - ps := st.(*stmts.PreparedStmt) - s.history.add(ps.ID, st) - } else { - s.history.add(0, st) - } - - if r != nil { - rs = append(rs, r) - } - } - - return rs, nil -} - -// For execute prepare statement in binary protocol -func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields []*field.ResultField, err error) { - return prepareStmt(s, sql) -} - -// checkArgs makes sure all the arguments' types are known and can be handled. -// integer types are converted to int64 and uint64, time.Time is converted to mysql.Time. -// time.Duration is converted to mysql.Duration, other known types are leaved as it is. -func checkArgs(args ...interface{}) error { - for i, v := range args { - switch x := v.(type) { - case bool: - if x { - args[i] = int64(1) - } else { - args[i] = int64(0) - } - case int8: - args[i] = int64(x) - case int16: - args[i] = int64(x) - case int32: - args[i] = int64(x) - case int: - args[i] = int64(x) - case uint8: - args[i] = uint64(x) - case uint16: - args[i] = uint64(x) - case uint32: - args[i] = uint64(x) - case uint: - args[i] = uint64(x) - case int64: - case uint64: - case float32: - case float64: - case string: - case []byte: - case time.Duration: - args[i] = mysql.Duration{Duration: x} - case time.Time: - args[i] = mysql.Time{Time: x, Type: mysql.TypeDatetime} - case nil: - default: - return errors.Errorf("cannot use arg[%d] (type %T):unsupported type", i, v) - } - } - return nil -} - -// Execute a prepared statement -func (s *session) ExecutePreparedStmt(stmtID uint32, args ...interface{}) (rset.Recordset, error) { - err := checkArgs(args...) - if err != nil { - return nil, err - } - - st := &stmts.ExecuteStmt{ID: stmtID} - s.history.add(stmtID, st, args...) - return runStmt(s, st, args...) -} - -func (s *session) DropPreparedStmt(stmtID uint32) error { - return dropPreparedStmt(s, stmtID) -} - -// If forceNew is true, GetTxn() must return a new transaction. -// In this situation, if current transaction is still in progress, -// there will be an implicit commit and create a new transaction. -func (s *session) GetTxn(forceNew bool) (kv.Transaction, error) { - var err error - if s.txn == nil { - s.resetHistory() - s.txn, err = s.store.Begin() - if err != nil { - return nil, err - } - if !variable.IsAutocommit(s) { - variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusInTrans, true) - } - log.Infof("New txn:%s in session:%d", s.txn, s.sid) - return s.txn, nil - } - if forceNew { - err = s.txn.Commit() - variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusInTrans, false) - if err != nil { - return nil, err - } - s.resetHistory() - s.txn, err = s.store.Begin() - if err != nil { - return nil, err - } - if !variable.IsAutocommit(s) { - variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusInTrans, true) - } - log.Warnf("Force new txn:%s in session:%d", s.txn, s.sid) - } - return s.txn, nil -} - -func (s *session) SetValue(key fmt.Stringer, value interface{}) { - s.values[key] = value -} - -func (s *session) Value(key fmt.Stringer) interface{} { - value := s.values[key] - return value -} - -func (s *session) ClearValue(key fmt.Stringer) { - delete(s.values, key) -} - -// Close function does some clean work when session end. -func (s *session) Close() error { - return s.FinishTxn(true) -} - -func (s *session) Auth(user string, auth []byte, salt []byte) bool { - strs := strings.Split(user, "@") - if len(strs) != 2 { - log.Warnf("Invalid format for user: %s", user) - return false - } - // Get user password. - name := strs[0] - host := strs[1] - authSQL := fmt.Sprintf("SELECT Password FROM %s.%s WHERE User=\"%s\" and Host=\"%s\";", mysql.SystemDB, mysql.UserTable, name, host) - rs, err := s.Execute(authSQL) - if err != nil { - log.Warnf("Encounter error when auth user %s. Error: %v", user, err) - return false - } - if len(rs) == 0 { - return false - } - row, err := rs[0].Next() - if err != nil { - log.Warnf("Encounter error when auth user %s. Error: %v", user, err) - return false - } - if row == nil || len(row.Data) == 0 { - return false - } - pwd, ok := row.Data[0].(string) - if !ok { - return false - } - hpwd, err := util.DecodePassword(pwd) - if err != nil { - log.Errorf("Decode password string error %v", err) - return false - } - checkAuth := util.CalcPassword(salt, hpwd) - if !bytes.Equal(auth, checkAuth) { - return false - } - variable.GetSessionVars(s).SetCurrentUser(user) - return true -} - -// CreateSession creates a new session environment. -func CreateSession(store kv.Storage) (Session, error) { - s := &session{ - values: make(map[fmt.Stringer]interface{}), - store: store, - sid: atomic.AddInt64(&sessionID, 1), - } - domain, err := domap.Get(store) - if err != nil { - return nil, err - } - sessionctx.BindDomain(s, domain) - - variable.BindSessionVars(s) - variable.GetSessionVars(s).SetStatusFlag(mysql.ServerStatusAutocommit, true) - sessionMu.Lock() - defer sessionMu.Unlock() - - _, ok := storeBootstrapped[store.UUID()] - if !ok { - bootstrap(s) - storeBootstrapped[store.UUID()] = true - } - // Add auth here - return s, nil -} From 2feb3d969c6914ed9fb7f0c25315911f5b5c7140 Mon Sep 17 00:00:00 2001 From: xia Date: Thu, 15 Oct 2015 19:54:13 +0800 Subject: [PATCH 38/58] tidb: add test --- tidb_test.go | 5 +++++ 1 file changed, 5 insertions(+) diff --git a/tidb_test.go b/tidb_test.go index 20ae252e28..44e3f6960c 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -807,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) From 1466a477aeed2531f195da0dc43d72910e4a5656 Mon Sep 17 00:00:00 2001 From: xia Date: Thu, 15 Oct 2015 19:50:19 +0800 Subject: [PATCH 39/58] parser: fix issue#340 and add test --- parser/parser.y | 23 ++++++++++++++++++----- parser/parser_test.go | 2 ++ 2 files changed, 20 insertions(+), 5 deletions(-) diff --git a/parser/parser.y b/parser/parser.y index 9ba09fc49d..c782bbb6c5 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -2932,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 @@ -2978,8 +2992,7 @@ SelectStmt: } FromDual: - /* Empty */ -| "FROM" "DUAL" + "FROM" "DUAL" FromClause: diff --git a/parser/parser_test.go b/parser/parser_test.go index 28fae5e330..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}, From a8779d8d528dfbcc8d3d6b71e618da0da7143297 Mon Sep 17 00:00:00 2001 From: Shen Li Date: Thu, 15 Oct 2015 22:50:55 +0800 Subject: [PATCH 40/58] *: Address comment --- session.go | 9 ++++++--- stmt/stmts/account_manage.go | 6 +++--- tidb_test.go | 11 +++++++++++ util/sqlhelper/sql_helper.go | 17 +++++++++++++++++ 4 files changed, 37 insertions(+), 6 deletions(-) diff --git a/session.go b/session.go index d2dbe25883..9a98d43668 100644 --- a/session.go +++ b/session.go @@ -234,15 +234,18 @@ func (s *session) Retry() error { // This is used for execute some restricted sql statement. func (s *session) ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recordset, error) { if ctx.Value(&sqlhelper.KeyType{}) != nil { - return nil, 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("Syntax error: %s", sql) return nil, errors.Trace(err) } - if len(statements) == 0 { - log.Warnf("No statement to exec for %s", sql) + 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 diff --git a/stmt/stmts/account_manage.go b/stmt/stmts/account_manage.go index 5bed6510cf..da7305e405 100644 --- a/stmt/stmts/account_manage.go +++ b/stmt/stmts/account_manage.go @@ -71,7 +71,7 @@ func (s *CreateUserStmt) SetText(text string) { } func (s *CreateUserStmt) userExists(ctx context.Context, name string, host string) (bool, error) { - sql := fmt.Sprintf("SELECT * FROM %s.%s WHERE User=\"%s\" AND HOST=\"%s\"", mysql.SystemDB, mysql.UserTable, name, host) + sql := fmt.Sprintf(`SELECT * FROM %s.%s WHERE User="%s" AND Host="%s";`, mysql.SystemDB, mysql.UserTable, name, host) rs, err := ctx.(sqlhelper.SQLHelper).ExecRestrictedSQL(ctx, sql) if err != nil { return false, errors.Trace(err) @@ -112,7 +112,7 @@ func (s *CreateUserStmt) Exec(ctx context.Context) (rset.Recordset, error) { if len(users) == 0 { return nil, nil } - sql := fmt.Sprintf("INSERT INTO %s.%s (Host, User, Password) VALUES %s;", mysql.SystemDB, mysql.UserTable, strings.Join(users, ", ")) + sql := fmt.Sprintf(`INSERT INTO %s.%s (Host, User, Password) VALUES %s;`, mysql.SystemDB, mysql.UserTable, strings.Join(users, ", ")) _, err := ctx.(sqlhelper.SQLHelper).ExecRestrictedSQL(ctx, sql) if err != nil { return nil, errors.Trace(err) @@ -156,7 +156,7 @@ func (s *SetPwdStmt) Exec(ctx context.Context) (rset.Recordset, error) { userName := strs[0] host := strs[1] // Update mysql.user - 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) + 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.(sqlhelper.SQLHelper).ExecRestrictedSQL(ctx, sql) return nil, errors.Trace(err) } diff --git a/tidb_test.go b/tidb_test.go index 84bdc17462..4ca12b77a7 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -1084,7 +1084,18 @@ func (s *testSessionSuite) TestDefaultFlenBug(c *C) { 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 newSession(c *C, store kv.Storage, dbName string) Session { diff --git a/util/sqlhelper/sql_helper.go b/util/sqlhelper/sql_helper.go index 8960a7a768..85b1d378cf 100644 --- a/util/sqlhelper/sql_helper.go +++ b/util/sqlhelper/sql_helper.go @@ -1,3 +1,20 @@ +// Copyright 2013 The ql Authors. All rights reserved. +// Use of this source code is governed by a BSD-style +// license that can be found in the LICENSES/QL-LICENSE file. + +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + package sqlhelper import ( From 67081336f287478123ef063ca076bdd8555386cd Mon Sep 17 00:00:00 2001 From: Shen Li Date: Thu, 15 Oct 2015 23:07:36 +0800 Subject: [PATCH 41/58] tidb: Address comments --- session.go | 7 ++----- 1 file changed, 2 insertions(+), 5 deletions(-) diff --git a/session.go b/session.go index 9a98d43668..0845dde14b 100644 --- a/session.go +++ b/session.go @@ -240,7 +240,7 @@ func (s *session) ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recor } statements, err := Compile(sql) if err != nil { - log.Errorf("Syntax error: %s", sql) + log.Errorf("Compile %s with error: %v", sql, err) return nil, errors.Trace(err) } if len(statements) != 1 { @@ -254,10 +254,7 @@ func (s *session) ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recor ctx.SetValue(&sqlhelper.KeyType{}, true) defer ctx.ClearValue(&sqlhelper.KeyType{}) rs, err := st.Exec(ctx) - if err != nil { - return nil, errors.Trace(err) - } - return rs, err + return rs, errors.Trace(err) } func (s *session) Execute(sql string) ([]rset.Recordset, error) { From 48976af41c701fe0756f76dfc37a8fb63a529c35 Mon Sep 17 00:00:00 2001 From: shenli Date: Fri, 16 Oct 2015 10:04:49 +0800 Subject: [PATCH 42/58] *: Address comments --- session.go | 2 +- stmt/stmts/account_manage.go | 2 +- util/sqlhelper/sql_helper.go | 4 ---- 3 files changed, 2 insertions(+), 6 deletions(-) diff --git a/session.go b/session.go index 0845dde14b..b25cd08edb 100644 --- a/session.go +++ b/session.go @@ -231,7 +231,7 @@ func (s *session) Retry() error { } // ExecRestrictedSQL implements SQLHelper interface. -// This is used for execute some restricted sql statement. +// This is used for executing some restricted sql statements. func (s *session) ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recordset, error) { if ctx.Value(&sqlhelper.KeyType{}) != nil { // We do not support run this function concurrently. diff --git a/stmt/stmts/account_manage.go b/stmt/stmts/account_manage.go index da7305e405..5a7f37f97b 100644 --- a/stmt/stmts/account_manage.go +++ b/stmt/stmts/account_manage.go @@ -106,7 +106,7 @@ func (s *CreateUserStmt) Exec(ctx context.Context) (rset.Recordset, error) { } else { pwd = util.EncodePassword(spec.AuthOpt.HashString) } - user := fmt.Sprintf("(\"%s\", \"%s\", \"%s\")", host, userName, pwd) + user := fmt.Sprintf(`("%s", "%s", "%s")`, host, userName, pwd) users = append(users, user) } if len(users) == 0 { diff --git a/util/sqlhelper/sql_helper.go b/util/sqlhelper/sql_helper.go index 85b1d378cf..ba998289b7 100644 --- a/util/sqlhelper/sql_helper.go +++ b/util/sqlhelper/sql_helper.go @@ -1,7 +1,3 @@ -// Copyright 2013 The ql Authors. All rights reserved. -// Use of this source code is governed by a BSD-style -// license that can be found in the LICENSES/QL-LICENSE file. - // Copyright 2015 PingCAP, Inc. // // Licensed under the Apache License, Version 2.0 (the "License"); From 08e0491b5a5d040a7800a517ed5c74ca159effb0 Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 08:56:20 +0800 Subject: [PATCH 43/58] *: having adds SelectList field. --- plan/plans/having.go | 1 + plan/plans/having_test.go | 4 ++++ rset/rsets/having.go | 7 ++++--- rset/rsets/having_test.go | 14 ++++++++------ stmt/stmts/select.go | 5 +++-- 5 files changed, 20 insertions(+), 11 deletions(-) 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/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/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 } } From 239fcabdf71aff4e1ded71f4e2be8fd3ec945c50 Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 11:29:29 +0800 Subject: [PATCH 44/58] plans: add check ambiguous helper function --- plan/plans/select_list.go | 54 +++++++++++++++- plan/plans/select_list_test.go | 111 +++++++++++++++++++++++++++++++++ 2 files changed, 164 insertions(+), 1 deletion(-) create mode 100644 plan/plans/select_list_test.go diff --git a/plan/plans/select_list.go b/plan/plans/select_list.go index d98abd47b4..1ad5e2e42f 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 } @@ -140,6 +143,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 strings.Contains(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 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.O != s.ResultFields[lastIndex].ColumnInfo.Name.O { + return -1, errors.Errorf("refer %s is ambiguous", expr) + } + + // check table name, e.g, "select t1.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) { 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) + } +} From 4957d281b5b756fbf096eb9ad3a0d1b69a21c8f0 Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 11:29:54 +0800 Subject: [PATCH 45/58] *: update group by ambiguous check --- rset/rsets/groupby.go | 55 +++++++------------------------------- rset/rsets/groupby_test.go | 42 ----------------------------- tidb_test.go | 30 +++++++++++++++++++++ 3 files changed, 40 insertions(+), 87 deletions(-) diff --git a/rset/rsets/groupby.go b/rset/rsets/groupby.go index ceee498980..0278dd8331 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,18 @@ 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: 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/tidb_test.go b/tidb_test.go index 20ae252e28..2b8a508d53 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -1116,6 +1116,24 @@ func (s *testSessionSuite) TestDefaultFlenBug(c *C) { } +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 newSession(c *C, store kv.Storage, dbName string) Session { se, err := CreateSession(store) c.Assert(err, IsNil) @@ -1187,3 +1205,15 @@ func matches(c *C, rows [][]interface{}, expected [][]interface{}) { 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) +} From 83fa8f8d4ff3b5c016155f9833b5caa06e542ebd Mon Sep 17 00:00:00 2001 From: shenli Date: Fri, 16 Oct 2015 12:26:10 +0800 Subject: [PATCH 46/58] *: Address comment --- session.go | 9 ++--- stmt/stmts/account_manage.go | 8 ++--- util/sqlexec/restricted_sql_executor.go | 44 +++++++++++++++++++++++++ util/sqlhelper/sql_helper.go | 33 ------------------- 4 files changed, 53 insertions(+), 41 deletions(-) create mode 100644 util/sqlexec/restricted_sql_executor.go delete mode 100644 util/sqlhelper/sql_helper.go diff --git a/session.go b/session.go index b25cd08edb..31c6a703a7 100644 --- a/session.go +++ b/session.go @@ -41,7 +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/sqlhelper" + "github.com/pingcap/tidb/util/sqlexec" ) // Session context @@ -233,7 +233,7 @@ func (s *session) Retry() error { // 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(&sqlhelper.KeyType{}) != nil { + 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.") @@ -250,9 +250,10 @@ func (s *session) ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recor 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(&sqlhelper.KeyType{}, true) - defer ctx.ClearValue(&sqlhelper.KeyType{}) + ctx.SetValue(&sqlexec.RestrictedSQLExecutorKeyType{}, true) + defer ctx.ClearValue(&sqlexec.RestrictedSQLExecutorKeyType{}) rs, err := st.Exec(ctx) return rs, errors.Trace(err) } diff --git a/stmt/stmts/account_manage.go b/stmt/stmts/account_manage.go index 5a7f37f97b..c40e8ebc5f 100644 --- a/stmt/stmts/account_manage.go +++ b/stmt/stmts/account_manage.go @@ -29,7 +29,7 @@ import ( "github.com/pingcap/tidb/stmt" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/format" - "github.com/pingcap/tidb/util/sqlhelper" + "github.com/pingcap/tidb/util/sqlexec" ) /************************************************************************************ @@ -72,7 +72,7 @@ func (s *CreateUserStmt) SetText(text string) { func (s *CreateUserStmt) userExists(ctx context.Context, name string, host string) (bool, error) { sql := fmt.Sprintf(`SELECT * FROM %s.%s WHERE User="%s" AND Host="%s";`, mysql.SystemDB, mysql.UserTable, name, host) - rs, err := ctx.(sqlhelper.SQLHelper).ExecRestrictedSQL(ctx, sql) + rs, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, sql) if err != nil { return false, errors.Trace(err) } @@ -113,7 +113,7 @@ func (s *CreateUserStmt) Exec(ctx context.Context) (rset.Recordset, error) { return nil, nil } sql := fmt.Sprintf(`INSERT INTO %s.%s (Host, User, Password) VALUES %s;`, mysql.SystemDB, mysql.UserTable, strings.Join(users, ", ")) - _, err := ctx.(sqlhelper.SQLHelper).ExecRestrictedSQL(ctx, sql) + _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, sql) if err != nil { return nil, errors.Trace(err) } @@ -157,6 +157,6 @@ func (s *SetPwdStmt) Exec(ctx context.Context) (rset.Recordset, error) { host := strs[1] // Update mysql.user 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.(sqlhelper.SQLHelper).ExecRestrictedSQL(ctx, sql) + _, err := ctx.(sqlexec.RestrictedSQLExecutor).ExecRestrictedSQL(ctx, sql) return nil, errors.Trace(err) } 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/sqlhelper/sql_helper.go b/util/sqlhelper/sql_helper.go deleted file mode 100644 index ba998289b7..0000000000 --- a/util/sqlhelper/sql_helper.go +++ /dev/null @@ -1,33 +0,0 @@ -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package sqlhelper - -import ( - "github.com/pingcap/tidb/context" - "github.com/pingcap/tidb/rset" -) - -// KeyType is a dummy type to avoid naming collision in session. -type KeyType struct{} - -// String implements Stringer interface. -func (k *KeyType) String() string { - return "sql_helper" -} - -// SQLHelper is an interface provides executing restricted sql statement. -type SQLHelper interface { - // ExecRestrictedSQL run sql statement in ctx with some restriction. - ExecRestrictedSQL(ctx context.Context, sql string) (rset.Recordset, error) -} From db09cf7a831250bde76e514cbaf6e01bb9f11941 Mon Sep 17 00:00:00 2001 From: ngaut Date: Fri, 16 Oct 2015 12:59:31 +0800 Subject: [PATCH 47/58] *: Tiny clean up --- kv/union_iter.go | 6 +++--- store/localstore/kv.go | 1 + store/localstore/local_version_provider.go | 1 + 3 files changed, 5 insertions(+), 3 deletions(-) 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/store/localstore/kv.go b/store/localstore/kv.go index 58934543fc..fbd326c5bb 100644 --- a/store/localstore/kv.go +++ b/store/localstore/kv.go @@ -139,6 +139,7 @@ func (s *dbStore) Begin() (kv.Transaction, error) { func (s *dbStore) Close() error { mc.mu.Lock() defer mc.mu.Unlock() + delete(mc.cache, s.path) return s.db.Close() } diff --git a/store/localstore/local_version_provider.go b/store/localstore/local_version_provider.go index f22ffd6778..34edfab2f5 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) { From b3ffb6e4613c1655a5d719b606649af414c2b581 Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 13:02:07 +0800 Subject: [PATCH 48/58] *: update order by ambiguous check --- rset/rsets/orderby.go | 24 ++++++++++++++++++------ tidb_test.go | 19 +++++++++++++++++++ 2 files changed, 37 insertions(+), 6 deletions(-) diff --git a/rset/rsets/orderby.go b/rset/rsets/orderby.go index 9a8ed3db13..1037f14976 100644 --- a/rset/rsets/orderby.go +++ b/rset/rsets/orderby.go @@ -76,16 +76,21 @@ 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: 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) - } + // // 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 +145,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/tidb_test.go b/tidb_test.go index 2b8a508d53..6c58eefd85 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -1134,6 +1134,25 @@ func (s *testSessionSuite) TestGroupBy(c *C) { 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}}) + + // 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 newSession(c *C, store kv.Storage, dbName string) Session { se, err := CreateSession(store) c.Assert(err, IsNil) From 9601ee3e846fed6748d644bf53b1a14e3293541b Mon Sep 17 00:00:00 2001 From: ngaut Date: Fri, 16 Oct 2015 13:03:32 +0800 Subject: [PATCH 49/58] typo --- kv/kv.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/kv/kv.go b/kv/kv.go index 6098f0f604..b93e9dc3a2 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 From 46b6b5f691c5087d51822f0942e4607aab34c245 Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 13:14:01 +0800 Subject: [PATCH 50/58] *: fix update arg field bug. --- plan/plans/select_list.go | 5 +++-- tidb_test.go | 12 ++++++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/plan/plans/select_list.go b/plan/plans/select_list.go index 1ad5e2e42f..c4a7007259 100644 --- a/plan/plans/select_list.go +++ b/plan/plans/select_list.go @@ -114,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) @@ -122,7 +123,7 @@ func (s *SelectList) UpdateAggFields(expr expression.Expression, tableFields []* return &expression.Position{N: len(s.Fields), Name: exprName}, nil } - return nil, nil + return &expression.Position{N: idx[0] + 1, Name: exprName}, nil } // CloneHiddenField checks and clones field and result field from table fields, diff --git a/tidb_test.go b/tidb_test.go index 6c58eefd85..e4f6d0cf44 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -1146,6 +1146,7 @@ func (s *testSessionSuite) TestOrderBy(c *C) { 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}}) @@ -1153,6 +1154,17 @@ func (s *testSessionSuite) TestOrderBy(c *C) { 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) From b21bd114f80fb71e385bc1e25c6799eb03607bc9 Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 13:25:09 +0800 Subject: [PATCH 51/58] *: fix ambiguous check --- plan/plans/select_list.go | 12 ++++++++++-- tidb_test.go | 2 ++ 2 files changed, 12 insertions(+), 2 deletions(-) diff --git a/plan/plans/select_list.go b/plan/plans/select_list.go index c4a7007259..8bb6535da7 100644 --- a/plan/plans/select_list.go +++ b/plan/plans/select_list.go @@ -233,8 +233,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/tidb_test.go b/tidb_test.go index e4f6d0cf44..452b4817b2 100644 --- a/tidb_test.go +++ b/tidb_test.go @@ -899,6 +899,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) { From 7c17ad0e52c02ccfeeadc5f06a10a591a7b0da5b Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 13:33:59 +0800 Subject: [PATCH 52/58] *: cleanup. --- plan/plans/select_list.go | 1 + rset/rsets/orderby.go | 4 ---- 2 files changed, 1 insertion(+), 4 deletions(-) diff --git a/plan/plans/select_list.go b/plan/plans/select_list.go index 8bb6535da7..be4fb1c67d 100644 --- a/plan/plans/select_list.go +++ b/plan/plans/select_list.go @@ -123,6 +123,7 @@ func (s *SelectList) UpdateAggFields(expr expression.Expression, tableFields []* return &expression.Position{N: len(s.Fields), Name: exprName}, nil } + // select list has this field, use it directly. return &expression.Position{N: idx[0] + 1, Name: exprName}, nil } diff --git a/rset/rsets/orderby.go b/rset/rsets/orderby.go index 1037f14976..0d613c463c 100644 --- a/rset/rsets/orderby.go +++ b/rset/rsets/orderby.go @@ -87,10 +87,6 @@ func (r *OrderByRset) CheckAndUpdateSelectList(selectList *plans.SelectList, tab // 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 } From 02f4a0a3b5080506542a9aec378b993347625bb2 Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 15:51:23 +0800 Subject: [PATCH 53/58] plans: fix comment typo --- plan/plans/select_list.go | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/plan/plans/select_list.go b/plan/plans/select_list.go index be4fb1c67d..4560773227 100644 --- a/plan/plans/select_list.go +++ b/plan/plans/select_list.go @@ -183,7 +183,7 @@ func (s *SelectList) CheckReferAmbiguous(expr expression.Expression) (int, error return -1, errors.Errorf("refer %s is ambiguous", expr) } - // check table name, e.g, "select t1.c1, c1 from t group by c1" is not ambiguous. + // 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) } From 24a64208006f80a957eb1e3af0b083162e4c4c8e Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 17:02:35 +0800 Subject: [PATCH 54/58] rsets: add comment. --- rset/rsets/groupby.go | 7 +++++++ rset/rsets/orderby.go | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/rset/rsets/groupby.go b/rset/rsets/groupby.go index 0278dd8331..aa3560bf97 100644 --- a/rset/rsets/groupby.go +++ b/rset/rsets/groupby.go @@ -77,6 +77,13 @@ func (r *GroupByRset) Plan(ctx context.Context) (plan.Plan, error) { 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 { diff --git a/rset/rsets/orderby.go b/rset/rsets/orderby.go index 0d613c463c..e0b9e74834 100644 --- a/rset/rsets/orderby.go +++ b/rset/rsets/orderby.go @@ -80,6 +80,13 @@ func (r *OrderByRset) CheckAndUpdateSelectList(selectList *plans.SelectList, tab 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 { From 551138e2477ce9c9289d5cae2ed48633fb9f7832 Mon Sep 17 00:00:00 2001 From: ngaut Date: Fri, 16 Oct 2015 17:40:21 +0800 Subject: [PATCH 55/58] store: Rename tID to tid --- store/localstore/kv.go | 12 ++++++------ store/localstore/txn.go | 20 ++++++++++---------- 2 files changed, 16 insertions(+), 16 deletions(-) diff --git a/store/localstore/kv.go b/store/localstore/kv.go index fbd326c5bb..0fc6566a81 100644 --- a/store/localstore/kv.go +++ b/store/localstore/kv.go @@ -119,13 +119,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, @@ -162,7 +162,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() @@ -185,12 +185,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/txn.go b/store/localstore/txn.go index ae315167c8..741a2c7154 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 { @@ -114,11 +114,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) { @@ -141,13 +141,13 @@ 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) return txn.UnionStore.Set(k, data) } 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) @@ -168,7 +168,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) } @@ -194,7 +194,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) } @@ -232,7 +232,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() }() @@ -259,7 +259,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() } From 7c05c9d9699e7e2dc9ce38dc75a8b3cd2d7b4a44 Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 19:07:53 +0800 Subject: [PATCH 56/58] *: use IsQualifiedName --- expression/helper.go | 5 ----- field/result_field.go | 5 +++++ plan/plans/select_list.go | 2 +- 3 files changed, 6 insertions(+), 6 deletions(-) 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/result_field.go b/field/result_field.go index c6847bc3db..2eb301b9d2 100644 --- a/field/result_field.go +++ b/field/result_field.go @@ -281,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/plan/plans/select_list.go b/plan/plans/select_list.go index 4560773227..89c809d9e4 100644 --- a/plan/plans/select_list.go +++ b/plan/plans/select_list.go @@ -157,7 +157,7 @@ func (s *SelectList) CheckReferAmbiguous(expr expression.Expression) (int, error } name := expr.String() - if strings.Contains(name, ".") { + if field.IsQualifiedName(name) { // name is qualified, no need to check return -1, nil } From 3a97fa2f1980d9fbdf690bb80bb5d23703bfb0c0 Mon Sep 17 00:00:00 2001 From: siddontang Date: Fri, 16 Oct 2015 19:23:43 +0800 Subject: [PATCH 57/58] plans: Address comment --- plan/plans/select_list.go | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/plan/plans/select_list.go b/plan/plans/select_list.go index 89c809d9e4..7d918fceed 100644 --- a/plan/plans/select_list.go +++ b/plan/plans/select_list.go @@ -165,7 +165,7 @@ func (s *SelectList) CheckReferAmbiguous(expr expression.Expression) (int, error lastIndex := -1 // only check origin select list, no hidden field. for i := 0; i < s.HiddenFieldOffset; i++ { - if s.ResultFields[i].Name != name { + if !strings.EqualFold(s.ResultFields[i].Name, name) { continue } else if _, ok := s.Fields[i].Expr.(*expression.Ident); !ok { // not identfier, no check @@ -179,7 +179,7 @@ func (s *SelectList) CheckReferAmbiguous(expr expression.Expression) (int, error } // check origin name, e,g. "select c1 as c2, c2 from t group by c2" is ambiguous. - if s.ResultFields[i].ColumnInfo.Name.O != s.ResultFields[lastIndex].ColumnInfo.Name.O { + if s.ResultFields[i].ColumnInfo.Name.L != s.ResultFields[lastIndex].ColumnInfo.Name.L { return -1, errors.Errorf("refer %s is ambiguous", expr) } From c7447005d5118bb693f59788effa364e761e292d Mon Sep 17 00:00:00 2001 From: ngaut Date: Sat, 17 Oct 2015 13:50:06 +0800 Subject: [PATCH 58/58] store: Tiny clean up --- store/localstore/kv.go | 1 - store/localstore/snapshot.go | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/store/localstore/kv.go b/store/localstore/kv.go index 0fc6566a81..0bab034074 100644 --- a/store/localstore/kv.go +++ b/store/localstore/kv.go @@ -101,7 +101,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, 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