diff --git a/ast/ast.go b/ast/ast.go index f226bcb8f609b..8032ef37ed34f 100644 --- a/ast/ast.go +++ b/ast/ast.go @@ -176,6 +176,9 @@ type Statement interface { // IsPrepared returns whether this statement is prepared statement. IsPrepared() bool + + // IsReadOnly returns if the statement is read only. For example: SelectStmt without lock. + IsReadOnly() bool } // Visitor visits a Node. diff --git a/ast/read_only_checker.go b/ast/read_only_checker.go new file mode 100644 index 0000000000000..aa0e49f02fccb --- /dev/null +++ b/ast/read_only_checker.go @@ -0,0 +1,61 @@ +// Copyright 2017 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 + +// IsReadOnly checks whether the input ast is readOnly. +func IsReadOnly(node Node) bool { + switch st := node.(type) { + case *SelectStmt: + if st.LockTp == SelectLockForUpdate { + return false + } + + checker := readOnlyChecker{ + readOnly: true, + } + + node.Accept(&checker) + return checker.readOnly + case *ExplainStmt: + return true + default: + return false + } +} + +// readOnlyChecker checks whether a query's ast is readonly, if it satisfied +// 1. selectstmt; +// 2. need not to set var; +// it is readonly statement. +type readOnlyChecker struct { + readOnly bool +} + +// Enter implements Visitor interface. +func (checker *readOnlyChecker) Enter(in Node) (out Node, skipChildren bool) { + switch node := in.(type) { + case *VariableExpr: + // like func rewriteVariable(), this stands for SetVar. + if !node.IsSystem && node.Value != nil { + checker.readOnly = false + return in, true + } + } + return in, false +} + +// Leave implements Visitor interface. +func (checker *readOnlyChecker) Leave(in Node) (out Node, ok bool) { + return in, checker.readOnly +} diff --git a/ast/read_only_checker_test.go b/ast/read_only_checker_test.go new file mode 100644 index 0000000000000..0b56b4d85d388 --- /dev/null +++ b/ast/read_only_checker_test.go @@ -0,0 +1,41 @@ +// Copyright 2017 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/check" +) + +var _ = Suite(&testCacheableSuite{}) + +type testCacheableSuite struct { +} + +func (s *testCacheableSuite) TestCacheable(c *C) { + // test non-SelectStmt + var stmt Node = &DeleteStmt{} + c.Assert(IsReadOnly(stmt), IsFalse) + + stmt = &InsertStmt{} + c.Assert(IsReadOnly(stmt), IsFalse) + + stmt = &UpdateStmt{} + c.Assert(IsReadOnly(stmt), IsFalse) + + stmt = &ExplainStmt{} + c.Assert(IsReadOnly(stmt), IsTrue) + + stmt = &ExplainStmt{} + c.Assert(IsReadOnly(stmt), IsTrue) +} diff --git a/executor/adapter.go b/executor/adapter.go index a1226749f1d91..0317b1899e012 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -113,6 +113,9 @@ type ExecStmt struct { ctx context.Context startTime time.Time isPreparedStmt bool + + // ReadOnly represents the statement is read-only. + ReadOnly bool } // OriginText implements ast.Statement interface. @@ -125,6 +128,11 @@ func (a *ExecStmt) IsPrepared() bool { return a.isPreparedStmt } +// IsReadOnly implements ast.Statement interface. +func (a *ExecStmt) IsReadOnly() bool { + return a.ReadOnly +} + // Exec implements the ast.Statement Exec interface. // This function builds an Executor from a plan. If the Executor doesn't return result, // like the INSERT, UPDATE statements, it executes in this function, if the Executor returns diff --git a/executor/compiler.go b/executor/compiler.go index 0ca4a9c525248..bcea287a39740 100644 --- a/executor/compiler.go +++ b/executor/compiler.go @@ -42,12 +42,31 @@ func (c *Compiler) Compile(ctx context.Context, stmtNode ast.StmtNode) (*ExecStm return nil, errors.Trace(err) } + // check whether the stmt is read only + readOnly := false + if e, ok := stmtNode.(*ast.ExecuteStmt); ok { + vars := ctx.GetSessionVars() + + if id, ok := vars.PreparedStmtNameToID[e.Name]; ok { + v := vars.PreparedStmts[id] + if v == nil { + return nil, errors.Trace(ErrStmtNotFound) + } + + prepared := v.(*Prepared) + readOnly = ast.IsReadOnly(prepared.Stmt) + } + } else { + readOnly = ast.IsReadOnly(stmtNode) + } + return &ExecStmt{ InfoSchema: infoSchema, Plan: finalPlan, Expensive: stmtCount(stmtNode, finalPlan, ctx.GetSessionVars().InRestrictedSQL), Cacheable: plan.Cacheable(stmtNode), Text: stmtNode.Text(), + ReadOnly: readOnly, }, nil } diff --git a/executor/prepared.go b/executor/prepared.go index cc23d75c78927..a06f2039a026b 100644 --- a/executor/prepared.go +++ b/executor/prepared.go @@ -318,12 +318,16 @@ func CompileExecutePreparedStmt(ctx context.Context, ID uint32, args ...interfac value := ast.NewValueExpr(val) execPlan.UsingVars[i] = &expression.Constant{Value: value.Datum, RetType: &value.Type} } + stmt := &ExecStmt{ InfoSchema: GetInfoSchema(ctx), Plan: execPlan, + ReadOnly: false, } + if prepared, ok := ctx.GetSessionVars().PreparedStmts[ID].(*Prepared); ok { stmt.Text = prepared.Stmt.Text() + stmt.ReadOnly = ast.IsReadOnly(prepared.Stmt) } return stmt } diff --git a/session.go b/session.go index 449c92c6e1a42..40783e9b5d508 100644 --- a/session.go +++ b/session.go @@ -407,6 +407,9 @@ func (s *session) retry(maxCnt int, infoSchemaChanged bool) error { s.sessionVars.RetryInfo.ResetOffset() for i, sr := range nh.history { st := sr.st + if st.IsReadOnly() { + continue + } txt := st.OriginText() if infoSchemaChanged { st, err = updateStatement(st, s, txt) @@ -694,6 +697,7 @@ func (s *session) Execute(sql string) (recordSets []ast.RecordSet, err error) { Plan: cacheValue.(*cache.SQLCacheValue).Plan, Expensive: cacheValue.(*cache.SQLCacheValue).Expensive, Text: stmtNode.Text(), + ReadOnly: ast.IsReadOnly(stmtNode), } s.PrepareTxnCtx()