Skip to content

Commit

Permalink
executor: print arguments in execute statement in log files (#7684) (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
jackysp authored and zz-jason committed Oct 25, 2018
1 parent 73692d1 commit 7fb086f
Show file tree
Hide file tree
Showing 10 changed files with 172 additions and 139 deletions.
8 changes: 8 additions & 0 deletions ast/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -184,6 +184,14 @@ func (n *DeallocateStmt) Accept(v Visitor) (Node, bool) {
return v.Leave(n)
}

// Prepared represents a prepared statement.
type Prepared struct {
Stmt StmtNode
Params []*ParamMarkerExpr
SchemaVersion int64
UseCache bool
}

// ExecuteStmt is a statement to execute PreparedStmt.
// See https://dev.mysql.com/doc/refman/5.7/en/execute.html
type ExecuteStmt struct {
Expand Down
2 changes: 1 addition & 1 deletion ddl/db_change_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -296,7 +296,7 @@ func (t *testExecInfo) compileSQL(idx int) (err error) {
ctx := context.TODO()
se.PrepareTxnCtx(ctx)
sctx := se.(sessionctx.Context)
if err = executor.ResetStmtCtx(sctx, c.rawStmt); err != nil {
if err = executor.ResetContextOfStmt(sctx, c.rawStmt); err != nil {
return errors.Trace(err)
}
c.stmt, err = compiler.Compile(ctx, c.rawStmt)
Expand Down
5 changes: 2 additions & 3 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -323,7 +323,6 @@ func (a *ExecStmt) buildExecutor(ctx sessionctx.Context) (Executor, error) {
if err != nil {
return nil, errors.Trace(err)
}
a.Text = executorExec.stmt.Text()
a.isPreparedStmt = true
a.Plan = executorExec.plan
e = executorExec.stmtExec
Expand All @@ -350,9 +349,9 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) {
if len(sql) > int(cfg.Log.QueryLogMaxLen) {
sql = fmt.Sprintf("%.*q(len:%d)", cfg.Log.QueryLogMaxLen, sql, len(a.Text))
}
sql = QueryReplacer.Replace(sql)

sessVars := a.Ctx.GetSessionVars()
sql = QueryReplacer.Replace(sql) + sessVars.GetExecuteArgumentsInfo()

connID := sessVars.ConnectionID
currentDB := sessVars.CurrentDB
var tableIDs, indexIDs string
Expand Down
110 changes: 110 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@ import (

"github.com/cznic/mathutil"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/domain"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
Expand All @@ -30,13 +31,16 @@ import (
"github.com/pingcap/tidb/mysql"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/admin"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/memory"
"github.com/pkg/errors"
log "github.com/sirupsen/logrus"
"golang.org/x/net/context"
Expand Down Expand Up @@ -1161,3 +1165,109 @@ func (e *UnionExec) Close() error {
e.resourcePools = nil
return errors.Trace(e.baseExecutor.Close())
}

// ResetContextOfStmt resets the StmtContext and session variables.
// Before every execution, we must clear statement context.
func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
vars := ctx.GetSessionVars()
sc := new(stmtctx.StatementContext)
sc.TimeZone = vars.Location()
sc.MemTracker = memory.NewTracker(s.Text(), vars.MemQuotaQuery)
switch config.GetGlobalConfig().OOMAction {
case config.OOMActionCancel:
sc.MemTracker.SetActionOnExceed(&memory.PanicOnExceed{})
case config.OOMActionLog:
sc.MemTracker.SetActionOnExceed(&memory.LogOnExceed{})
default:
sc.MemTracker.SetActionOnExceed(&memory.LogOnExceed{})
}

if execStmt, ok := s.(*ast.ExecuteStmt); ok {
s, err = getPreparedStmt(execStmt, vars)
}
// TODO: Many same bool variables here.
// We should set only two variables (
// IgnoreErr and StrictSQLMode) to avoid setting the same bool variables and
// pushing them down to TiKV as flags.
switch stmt := s.(type) {
case *ast.UpdateStmt:
sc.InUpdateOrDeleteStmt = true
sc.DupKeyAsWarning = stmt.IgnoreErr
sc.BadNullAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.IgnoreZeroInDate = !vars.StrictSQLMode || stmt.IgnoreErr
sc.Priority = stmt.Priority
case *ast.DeleteStmt:
sc.InUpdateOrDeleteStmt = true
sc.DupKeyAsWarning = stmt.IgnoreErr
sc.BadNullAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.IgnoreZeroInDate = !vars.StrictSQLMode || stmt.IgnoreErr
sc.Priority = stmt.Priority
case *ast.InsertStmt:
sc.InInsertStmt = true
sc.DupKeyAsWarning = stmt.IgnoreErr
sc.BadNullAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.TruncateAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.DividedByZeroAsWarning = !vars.StrictSQLMode || stmt.IgnoreErr
sc.IgnoreZeroInDate = !vars.StrictSQLMode || stmt.IgnoreErr
sc.Priority = stmt.Priority
case *ast.CreateTableStmt, *ast.AlterTableStmt:
// Make sure the sql_mode is strict when checking column default value.
case *ast.LoadDataStmt:
sc.DupKeyAsWarning = true
sc.BadNullAsWarning = true
sc.TruncateAsWarning = !vars.StrictSQLMode
case *ast.SelectStmt:
sc.InSelectStmt = true

// see https://dev.mysql.com/doc/refman/5.7/en/sql-mode.html#sql-mode-strict
// said "For statements such as SELECT that do not change data, invalid values
// generate a warning in strict mode, not an error."
// and https://dev.mysql.com/doc/refman/5.7/en/out-of-range-and-overflow.html
sc.OverflowAsWarning = true

// Return warning for truncate error in selection.
sc.TruncateAsWarning = true
sc.IgnoreZeroInDate = true
if opts := stmt.SelectStmtOpts; opts != nil {
sc.Priority = opts.Priority
sc.NotFillCache = !opts.SQLCache
}
sc.PadCharToFullLength = ctx.GetSessionVars().SQLMode.HasPadCharToFullLengthMode()
case *ast.ShowStmt:
sc.IgnoreTruncate = true
sc.IgnoreZeroInDate = true
if stmt.Tp == ast.ShowWarnings || stmt.Tp == ast.ShowErrors {
sc.InShowWarning = true
sc.SetWarnings(vars.StmtCtx.GetWarnings())
}
default:
sc.IgnoreTruncate = true
sc.IgnoreZeroInDate = true
}
vars.PreparedParams = vars.PreparedParams[:0]
if !vars.InRestrictedSQL {
if priority := mysql.PriorityEnum(atomic.LoadInt32(&variable.ForcePriority)); priority != mysql.NoPriority {
sc.Priority = priority
}
}
if vars.LastInsertID > 0 {
vars.PrevLastInsertID = vars.LastInsertID
vars.LastInsertID = 0
}
vars.ResetPrevAffectedRows()
err = vars.SetSystemVar("warning_count", fmt.Sprintf("%d", vars.StmtCtx.NumWarnings(false)))
if err != nil {
return errors.Trace(err)
}
err = vars.SetSystemVar("error_count", fmt.Sprintf("%d", vars.StmtCtx.NumWarnings(true)))
if err != nil {
return errors.Trace(err)
}
vars.InsertID = 0
vars.StmtCtx = sc
return
}
122 changes: 14 additions & 108 deletions executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,22 +16,16 @@ package executor
import (
"math"
"sort"
"sync/atomic"

"fmt"
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/parser"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pkg/errors"
"golang.org/x/net/context"
Expand Down Expand Up @@ -152,7 +146,7 @@ func (e *PrepareExec) Next(ctx context.Context, chk *chunk.Chunk) error {
for i := 0; i < e.ParamCount; i++ {
sorter.markers[i].Order = i
}
prepared := &plannercore.Prepared{
prepared := &ast.Prepared{
Stmt: stmt,
Params: sorter.markers,
SchemaVersion: e.is.SchemaMetaVersion(),
Expand Down Expand Up @@ -219,9 +213,6 @@ func (e *ExecuteExec) Build() error {
return errors.Trace(b.err)
}
e.stmtExec = stmtExec
if err = ResetStmtCtx(e.ctx, e.stmt); err != nil {
return err
}
CountStmtNode(e.stmt, e.ctx.GetSessionVars().InRestrictedSQL)
logExpensiveQuery(e.stmt, e.plan)
return nil
Expand Down Expand Up @@ -249,6 +240,9 @@ func (e *DeallocateExec) Next(ctx context.Context, chk *chunk.Chunk) error {
// CompileExecutePreparedStmt compiles a session Execute command to a stmt.Statement.
func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...interface{}) (ast.Statement, error) {
execStmt := &ast.ExecuteStmt{ExecID: ID}
if err := ResetContextOfStmt(ctx, execStmt); err != nil {
return nil, err
}
execStmt.UsingVars = make([]ast.ExprNode, len(args))
for i, val := range args {
execStmt.UsingVars[i] = ast.NewValueExpr(val)
Expand All @@ -265,110 +259,22 @@ func CompileExecutePreparedStmt(ctx sessionctx.Context, ID uint32, args ...inter
StmtNode: execStmt,
Ctx: ctx,
}
if prepared, ok := ctx.GetSessionVars().PreparedStmts[ID].(*plannercore.Prepared); ok {
if prepared, ok := ctx.GetSessionVars().PreparedStmts[ID]; ok {
stmt.Text = prepared.Stmt.Text()
}
return stmt, nil
}

// ResetStmtCtx resets the StmtContext.
// Before every execution, we must clear statement context.
func ResetStmtCtx(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sessVars := ctx.GetSessionVars()
sc := new(stmtctx.StatementContext)
sc.TimeZone = sessVars.Location()
sc.MemTracker = memory.NewTracker(s.Text(), sessVars.MemQuotaQuery)
switch config.GetGlobalConfig().OOMAction {
case config.OOMActionCancel:
sc.MemTracker.SetActionOnExceed(&memory.PanicOnExceed{})
case config.OOMActionLog:
sc.MemTracker.SetActionOnExceed(&memory.LogOnExceed{})
default:
sc.MemTracker.SetActionOnExceed(&memory.LogOnExceed{})
}

// TODO: Many same bool variables here.
// We should set only two variables (
// IgnoreErr and StrictSQLMode) to avoid setting the same bool variables and
// pushing them down to TiKV as flags.
switch stmt := s.(type) {
case *ast.UpdateStmt:
sc.InUpdateOrDeleteStmt = true
sc.DupKeyAsWarning = stmt.IgnoreErr
sc.BadNullAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.TruncateAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.DividedByZeroAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.IgnoreZeroInDate = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.Priority = stmt.Priority
case *ast.DeleteStmt:
sc.InUpdateOrDeleteStmt = true
sc.DupKeyAsWarning = stmt.IgnoreErr
sc.BadNullAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.TruncateAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.DividedByZeroAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.IgnoreZeroInDate = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.Priority = stmt.Priority
case *ast.InsertStmt:
sc.InInsertStmt = true
sc.DupKeyAsWarning = stmt.IgnoreErr
sc.BadNullAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.TruncateAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.DividedByZeroAsWarning = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.IgnoreZeroInDate = !sessVars.StrictSQLMode || stmt.IgnoreErr
sc.Priority = stmt.Priority
case *ast.CreateTableStmt, *ast.AlterTableStmt:
// Make sure the sql_mode is strict when checking column default value.
case *ast.LoadDataStmt:
sc.DupKeyAsWarning = true
sc.BadNullAsWarning = true
sc.TruncateAsWarning = !sessVars.StrictSQLMode
case *ast.SelectStmt:
sc.InSelectStmt = true

// see https://dev.mysql.com/doc/refman/5.7/en/sql-mode.html#sql-mode-strict
// said "For statements such as SELECT that do not change data, invalid values
// generate a warning in strict mode, not an error."
// and https://dev.mysql.com/doc/refman/5.7/en/out-of-range-and-overflow.html
sc.OverflowAsWarning = true

// Return warning for truncate error in selection.
sc.TruncateAsWarning = true
sc.IgnoreZeroInDate = true
if opts := stmt.SelectStmtOpts; opts != nil {
sc.Priority = opts.Priority
sc.NotFillCache = !opts.SQLCache
func getPreparedStmt(stmt *ast.ExecuteStmt, vars *variable.SessionVars) (ast.StmtNode, error) {
execID := stmt.ExecID
ok := false
if stmt.Name != "" {
if execID, ok = vars.PreparedStmtNameToID[stmt.Name]; !ok {
return nil, plannercore.ErrStmtNotFound
}
sc.PadCharToFullLength = ctx.GetSessionVars().SQLMode.HasPadCharToFullLengthMode()
case *ast.ShowStmt:
sc.IgnoreTruncate = true
sc.IgnoreZeroInDate = true
if stmt.Tp == ast.ShowWarnings || stmt.Tp == ast.ShowErrors {
sc.InShowWarning = true
sc.SetWarnings(sessVars.StmtCtx.GetWarnings())
}
default:
sc.IgnoreTruncate = true
sc.IgnoreZeroInDate = true
}
if !sessVars.InRestrictedSQL {
if priority := mysql.PriorityEnum(atomic.LoadInt32(&variable.ForcePriority)); priority != mysql.NoPriority {
sc.Priority = priority
}
}
if sessVars.LastInsertID > 0 {
sessVars.PrevLastInsertID = sessVars.LastInsertID
sessVars.LastInsertID = 0
}
sessVars.ResetPrevAffectedRows()
err = sessVars.SetSystemVar("warning_count", fmt.Sprintf("%d", sessVars.StmtCtx.NumWarnings(false)))
if err != nil {
return errors.Trace(err)
}
err = sessVars.SetSystemVar("error_count", fmt.Sprintf("%d", sessVars.StmtCtx.NumWarnings(true)))
if err != nil {
return errors.Trace(err)
if prepared, ok := vars.PreparedStmts[execID]; ok {
return prepared.Stmt, nil
}
sessVars.InsertID = 0
sessVars.StmtCtx = sc
return
return nil, plannercore.ErrStmtNotFound
}
3 changes: 1 addition & 2 deletions expression/builtin_other.go
Original file line number Diff line number Diff line change
Expand Up @@ -771,8 +771,7 @@ func (b *builtinGetParamStringSig) evalString(row chunk.Row) (string, bool, erro
}
v := sessionVars.PreparedParams[idx]

dt := v.(types.Datum)
str, err := (&dt).ToString()
str, err := v.ToString()
if err != nil {
return "", true, nil
}
Expand Down
Loading

0 comments on commit 7fb086f

Please sign in to comment.