Skip to content

Commit

Permalink
Merge pull request pingcap#901 from pingcap/coocood/simple
Browse files Browse the repository at this point in the history
executor, optimizer: add simple plan and executor, implement UseStmt.
  • Loading branch information
coocood committed Feb 16, 2016
2 parents fa7e3da + db2328b commit 8c7d7d2
Show file tree
Hide file tree
Showing 14 changed files with 310 additions and 574 deletions.
6 changes: 6 additions & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,8 @@ func (b *executorBuilder) build(p plan.Plan) Executor {
return b.buildSelectLock(v)
case *plan.ShowDDL:
return b.buildShowDDL(v)
case *plan.Simple:
return b.buildSimple(v)
case *plan.Sort:
return b.buildSort(v)
case *plan.TableScan:
Expand Down Expand Up @@ -327,3 +329,7 @@ func (b *executorBuilder) buildDelete(v *plan.Delete) Executor {
IsMultiTable: v.IsMultiTable,
}
}

func (b *executorBuilder) buildSimple(v *plan.Simple) Executor {
return &SimpleExec{Statement: v.Statement, ctx: b.ctx}
}
45 changes: 0 additions & 45 deletions executor/converter/convert_stmt.go
Original file line number Diff line number Diff line change
Expand Up @@ -868,51 +868,6 @@ func convertRollback(converter *expressionConverter, v *ast.RollbackStmt) (*stmt
}, nil
}

func convertUse(converter *expressionConverter, v *ast.UseStmt) (*stmts.UseStmt, error) {
return &stmts.UseStmt{
DBName: v.DBName,
Text: v.Text(),
}, nil
}

func convertVariableAssignment(converter *expressionConverter, v *ast.VariableAssignment) (*stmts.VariableAssignment, error) {
oldValue, err := convertExpr(converter, v.Value)
if err != nil {
return nil, errors.Trace(err)
}

return &stmts.VariableAssignment{
IsGlobal: v.IsGlobal,
IsSystem: v.IsSystem,
Name: v.Name,
Value: oldValue,
Text: v.Text(),
}, nil
}

func convertSet(converter *expressionConverter, v *ast.SetStmt) (*stmts.SetStmt, error) {
oldSet := &stmts.SetStmt{
Text: v.Text(),
Variables: make([]*stmts.VariableAssignment, len(v.Variables)),
}
for i, val := range v.Variables {
oldAssign, err := convertVariableAssignment(converter, val)
if err != nil {
return nil, errors.Trace(err)
}
oldSet.Variables[i] = oldAssign
}
return oldSet, nil
}

func convertSetCharset(converter *expressionConverter, v *ast.SetCharsetStmt) (*stmts.SetCharsetStmt, error) {
return &stmts.SetCharsetStmt{
Charset: v.Charset,
Collate: v.Collate,
Text: v.Text(),
}, nil
}

func convertSetPwd(converter *expressionConverter, v *ast.SetPwdStmt) (*stmts.SetPwdStmt, error) {
return &stmts.SetPwdStmt{
User: v.User,
Expand Down
6 changes: 0 additions & 6 deletions executor/converter/converter.go
Original file line number Diff line number Diff line change
Expand Up @@ -62,12 +62,8 @@ func (con *Converter) Convert(node ast.Node) (stmt.Statement, error) {
return convertRollback(c, v)
case *ast.SelectStmt:
return convertSelect(c, v)
case *ast.SetCharsetStmt:
return convertSetCharset(c, v)
case *ast.SetPwdStmt:
return convertSetPwd(c, v)
case *ast.SetStmt:
return convertSet(c, v)
case *ast.ShowStmt:
return convertShow(c, v)
case *ast.TruncateTableStmt:
Expand All @@ -76,8 +72,6 @@ func (con *Converter) Convert(node ast.Node) (stmt.Statement, error) {
return convertUnion(c, v)
case *ast.UpdateStmt:
return convertUpdate(c, v)
case *ast.UseStmt:
return convertUse(c, v)
}
return nil, nil
}
165 changes: 165 additions & 0 deletions executor/executor_simple.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,165 @@
// Copyright 2016 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor

import (
"fmt"
"strings"

"github.com/juju/errors"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/optimizer/evaluator"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/db"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/charset"
"github.com/pingcap/tidb/util/types"
)

// SimpleExec represents simple statement executor.
// For statements do simple execution.
// includes `UseStmt`, 'SetStmt`, `SetCharsetStmt`.
// TODO: list all simple statements.
type SimpleExec struct {
Statement ast.StmtNode
ctx context.Context
done bool
}

// Fields implements Executor Fields interface.
func (e *SimpleExec) Fields() []*ast.ResultField {
return nil
}

// Next implements Execution Next interface.
func (e *SimpleExec) Next() (*Row, error) {
if e.done {
return nil, nil
}
var err error
switch x := e.Statement.(type) {
case *ast.UseStmt:
err = e.executeUse(x)
case *ast.SetStmt:
err = e.executeSet(x)
case *ast.SetCharsetStmt:
err = e.executeSetCharset(x)
}
if err != nil {
return nil, errors.Trace(err)
}
e.done = true
return nil, nil
}

// Close implements Executor Close interface.
func (e *SimpleExec) Close() error {
return nil
}

func (e *SimpleExec) executeUse(s *ast.UseStmt) error {
dbname := model.NewCIStr(s.DBName)
dbinfo, exists := sessionctx.GetDomain(e.ctx).InfoSchema().SchemaByName(dbname)
if !exists {
return infoschema.DatabaseNotExists.Gen("database %s not exists", dbname)
}
db.BindCurrentSchema(e.ctx, dbname.O)
// character_set_database is the character set used by the default database.
// The server sets this variable whenever the default database changes.
// See: http://dev.mysql.com/doc/refman/5.7/en/server-system-variables.html#sysvar_character_set_database
sessionVars := variable.GetSessionVars(e.ctx)
sessionVars.Systems[variable.CharsetDatabase] = dbinfo.Charset
sessionVars.Systems[variable.CollationDatabase] = dbinfo.Collate
return nil
}

func (e *SimpleExec) executeSet(s *ast.SetStmt) error {
sessionVars := variable.GetSessionVars(e.ctx)
globalVars := variable.GetGlobalVarAccessor(e.ctx)
for _, v := range s.Variables {
// Variable is case insensitive, we use lower case.
name := strings.ToLower(v.Name)
if !v.IsSystem {
// User variable.
value, err := evaluator.Eval(e.ctx, v.Value)
if err != nil {
return errors.Trace(err)
}

if value == nil {
delete(sessionVars.Users, name)
} else {
sessionVars.Users[name] = fmt.Sprintf("%v", value)
}
return nil
}
sysVar := variable.GetSysVar(name)
if sysVar == nil {
return variable.UnknownSystemVar.Gen("Unknown system variable '%s'", name)
}
if sysVar.Scope == variable.ScopeNone {
return errors.Errorf("Variable '%s' is a read only variable", name)
}
if v.IsGlobal {
if sysVar.Scope&variable.ScopeGlobal > 0 {
value, err := evaluator.Eval(e.ctx, v.Value)
if err != nil {
return errors.Trace(err)
}
if value == nil {
value = ""
}
svalue, err := types.ToString(value)
if err != nil {
return errors.Trace(err)
}
err = globalVars.SetGlobalSysVar(e.ctx, name, svalue)
return errors.Trace(err)
}
return errors.Errorf("Variable '%s' is a SESSION variable and can't be used with SET GLOBAL", name)
}
if sysVar.Scope&variable.ScopeSession > 0 {
if value, err := evaluator.Eval(e.ctx, v.Value); err != nil {
return errors.Trace(err)
} else if value == nil {
sessionVars.Systems[name] = ""
} else {
sessionVars.Systems[name] = fmt.Sprintf("%v", value)
}
return nil
}
return errors.Errorf("Variable '%s' is a GLOBAL variable and should be set with SET GLOBAL", name)
}
return nil
}

func (e *SimpleExec) executeSetCharset(s *ast.SetCharsetStmt) error {
collation := s.Collate
if len(collation) == 0 {
var err error
collation, err = charset.GetDefaultCollation(s.Charset)
if err != nil {
return errors.Trace(err)
}
}
sessionVars := variable.GetSessionVars(e.ctx)
for _, v := range variable.SetNamesVariables {
sessionVars.Systems[v] = s.Charset
}
sessionVars.Systems[variable.CollationConnection] = collation
return nil
}
100 changes: 100 additions & 0 deletions executor/executor_simple_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,100 @@
// Copyright 2016 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package executor_test

import (
. "github.com/pingcap/check"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/util/testkit"
)

func (s *testSuite) TestCharsetDatabase(c *C) {
tk := testkit.NewTestKit(c, s.store)
testSQL := `create database if not exists cd_test_utf8 CHARACTER SET utf8 COLLATE utf8_bin;`
tk.MustExec(testSQL)

testSQL = `create database if not exists cd_test_latin1 CHARACTER SET latin1 COLLATE latin1_swedish_ci;`
tk.MustExec(testSQL)

testSQL = `use cd_test_utf8;`
tk.MustExec(testSQL)
tk.MustQuery(`select @@character_set_database;`).Check(testkit.Rows("utf8"))
tk.MustQuery(`select @@collation_database;`).Check(testkit.Rows("utf8_bin"))

testSQL = `use cd_test_latin1;`
tk.MustExec(testSQL)
tk.MustQuery(`select @@character_set_database;`).Check(testkit.Rows("latin1"))
tk.MustQuery(`select @@collation_database;`).Check(testkit.Rows("latin1_swedish_ci"))
}

func (s *testSuite) TestSet(c *C) {
tk := testkit.NewTestKit(c, s.store)
testSQL := "SET @a = 1;"
tk.MustExec(testSQL)

testSQL = `SET @a = "1";`
tk.MustExec(testSQL)

testSQL = "SET @a = null;"
tk.MustExec(testSQL)

testSQL = "SET @@global.autocommit = 1;"
tk.MustExec(testSQL)

testSQL = "SET @@global.autocommit = null;"
tk.MustExec(testSQL)

testSQL = "SET @@autocommit = 1;"
tk.MustExec(testSQL)

testSQL = "SET @@autocommit = null;"
tk.MustExec(testSQL)

errTestSql := "SET @@date_format = 1;"
_, err := tk.Exec(errTestSql)
c.Assert(err, NotNil)

errTestSql = "SET @@rewriter_enabled = 1;"
_, err = tk.Exec(errTestSql)
c.Assert(err, NotNil)

errTestSql = "SET xxx = abcd;"
_, err = tk.Exec(errTestSql)
c.Assert(err, NotNil)

errTestSql = "SET @@global.a = 1;"
_, err = tk.Exec(errTestSql)
c.Assert(err, NotNil)

errTestSql = "SET @@global.timestamp = 1;"
_, err = tk.Exec(errTestSql)
c.Assert(err, NotNil)
}

func (s *testSuite) TestSetCharset(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec(`SET NAMES latin1`)

ctx := tk.Se.(context.Context)
sessionVars := variable.GetSessionVars(ctx)
for _, v := range variable.SetNamesVariables {
c.Assert(sessionVars.Systems[v] != "utf8", IsTrue)
}
tk.MustExec(`SET NAMES utf8`)
for _, v := range variable.SetNamesVariables {
c.Assert(sessionVars.Systems[v], Equals, "utf8")
}
c.Assert(sessionVars.Systems[variable.CollationConnection], Equals, "utf8_general_ci")
}
1 change: 1 addition & 0 deletions optimizer/optimizer.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@ func IsSupported(node ast.Node) bool {
switch node.(type) {
case *ast.SelectStmt, *ast.PrepareStmt, *ast.ExecuteStmt, *ast.DeallocateStmt,
*ast.AdminStmt, *ast.UpdateStmt, *ast.DeleteStmt, *ast.UnionStmt:
case *ast.UseStmt, *ast.SetStmt, *ast.SetCharsetStmt:
default:
return false
}
Expand Down
10 changes: 10 additions & 0 deletions optimizer/plan/planbuilder.go
Original file line number Diff line number Diff line change
Expand Up @@ -71,6 +71,12 @@ func (b *planBuilder) build(node ast.Node) Plan {
return b.buildUnion(x)
case *ast.UpdateStmt:
return b.buildUpdate(x)
case *ast.UseStmt:
return b.buildSimple(x)
case *ast.SetCharsetStmt:
return b.buildSimple(x)
case *ast.SetStmt:
return b.buildSimple(x)
}
b.err = ErrUnsupportedType.Gen("Unsupported type %T", node)
return nil
Expand Down Expand Up @@ -790,3 +796,7 @@ func columnOffsetInFields(cn *ast.ColumnName, fields []*ast.ResultField) (int, e
}
return offset, nil
}

func (b *planBuilder) buildSimple(node ast.StmtNode) Plan {
return &Simple{Statement: node}
}
Loading

0 comments on commit 8c7d7d2

Please sign in to comment.