Skip to content

Commit

Permalink
Revert "tidb, variable: refactor retry, add TransactionContext" (ping…
Browse files Browse the repository at this point in the history
  • Loading branch information
hanfei1991 authored and shenli committed Dec 15, 2016
1 parent ff0ceb1 commit d406c77
Show file tree
Hide file tree
Showing 16 changed files with 224 additions and 110 deletions.
4 changes: 1 addition & 3 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -133,7 +133,6 @@ func (b *executorBuilder) buildCheckTable(v *plan.CheckTable) Executor {
return &CheckTableExec{
tables: v.Tables,
ctx: b.ctx,
is: b.is,
}
}

Expand Down Expand Up @@ -221,7 +220,7 @@ func (b *executorBuilder) buildSimple(v *plan.Simple) Executor {
case *ast.GrantStmt:
return b.buildGrant(s)
}
return &SimpleExec{Statement: v.Statement, ctx: b.ctx, is: b.is}
return &SimpleExec{Statement: v.Statement, ctx: b.ctx}
}

func (b *executorBuilder) buildSet(v *plan.Set) Executor {
Expand Down Expand Up @@ -306,7 +305,6 @@ func (b *executorBuilder) buildGrant(grant *ast.GrantStmt) Executor {
ObjectType: grant.ObjectType,
Level: grant.Level,
Users: grant.Users,
is: b.is,
}
}

Expand Down
26 changes: 11 additions & 15 deletions executor/compiler.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,8 @@ import (
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/binloginfo"
)

// Compiler compiles an ast.StmtNode to a stmt.Statement.
Expand Down Expand Up @@ -152,7 +154,15 @@ func getSelectStmtLabel(x *ast.SelectStmt) string {
// then wrappped to an adapter *statement as stmt.Statement.
func (c *Compiler) Compile(ctx context.Context, node ast.StmtNode) (ast.Statement, error) {
stmtCount(node)
is := GetInfoSchema(ctx)
var is infoschema.InfoSchema
sessVar := ctx.GetSessionVars()
if snap := sessVar.SnapshotInfoschema; snap != nil {
is = snap.(infoschema.InfoSchema)
log.Infof("[%d] use snapshot schema %d", sessVar.ConnectionID, is.SchemaMetaVersion())
} else {
is = sessionctx.GetDomain(ctx).InfoSchema()
binloginfo.SetSchemaVersion(ctx, is.SchemaMetaVersion())
}
if err := plan.Preprocess(node, is, ctx); err != nil {
return nil, errors.Trace(err)
}
Expand All @@ -173,17 +183,3 @@ func (c *Compiler) Compile(ctx context.Context, node ast.StmtNode) (ast.Statemen
}
return sa, nil
}

// GetInfoSchema gets TxnCtx InfoSchema if snapshot schema is not set,
// Otherwise, snapshot schema is returned.
func GetInfoSchema(ctx context.Context) infoschema.InfoSchema {
sessVar := ctx.GetSessionVars()
var is infoschema.InfoSchema
if snap := sessVar.SnapshotInfoschema; snap != nil {
is = snap.(infoschema.InfoSchema)
log.Infof("[%d] use snapshot schema %d", sessVar.ConnectionID, is.SchemaMetaVersion())
} else {
is = sessVar.TxnCtx.InfoSchema.(infoschema.InfoSchema)
}
return is
}
8 changes: 5 additions & 3 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -29,6 +29,8 @@ import (
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/forupdate"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/tablecodec"
Expand Down Expand Up @@ -188,7 +190,6 @@ type CheckTableExec struct {
tables []*ast.TableName
ctx context.Context
done bool
is infoschema.InfoSchema
}

// Schema implements the Executor Schema interface.
Expand All @@ -203,9 +204,10 @@ func (e *CheckTableExec) Next() (*Row, error) {
}

dbName := model.NewCIStr(e.ctx.GetSessionVars().CurrentDB)
is := sessionctx.GetDomain(e.ctx).InfoSchema()

for _, t := range e.tables {
tb, err := e.is.TableByName(dbName, t.Name)
tb, err := is.TableByName(dbName, t.Name)
if err != nil {
return nil, errors.Trace(err)
}
Expand Down Expand Up @@ -258,7 +260,7 @@ func (e *SelectLockExec) Next() (*Row, error) {
return nil, nil
}
if len(row.RowKeys) != 0 && e.Lock == ast.SelectLockForUpdate {
e.ctx.GetSessionVars().TxnCtx.ForUpdate = true
forupdate.SetForUpdate(e.ctx)
txn, err := e.ctx.GetTxn(false)
if err != nil {
return nil, errors.Trace(err)
Expand Down
4 changes: 2 additions & 2 deletions executor/executor_simple.go
Original file line number Diff line number Diff line change
Expand Up @@ -28,6 +28,7 @@ import (
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/plan/statistics"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/terror"
"github.com/pingcap/tidb/util"
Expand All @@ -44,7 +45,6 @@ type SimpleExec struct {
Statement ast.StmtNode
ctx context.Context
done bool
is infoschema.InfoSchema
}

// Schema implements the Executor Schema interface.
Expand Down Expand Up @@ -97,7 +97,7 @@ func (e *SimpleExec) Close() error {

func (e *SimpleExec) executeUse(s *ast.UseStmt) error {
dbname := model.NewCIStr(s.DBName)
dbinfo, exists := e.is.SchemaByName(dbname)
dbinfo, exists := sessionctx.GetDomain(e.ctx).InfoSchema().SchemaByName(dbname)
if !exists {
return infoschema.ErrDatabaseNotExists.GenByArgs(dbname)
}
Expand Down
1 change: 0 additions & 1 deletion executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1400,7 +1400,6 @@ func (s *testSuite) TestAdapterStatement(c *C) {
c.Check(err, IsNil)
compiler := &executor.Compiler{}
ctx := se.(context.Context)
tidb.PrepareTxnCtx(ctx)

stmtNode, err := s.ParseOneStmt("select 1", "", "")
c.Check(err, IsNil)
Expand Down
9 changes: 5 additions & 4 deletions executor/grant.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,9 +21,9 @@ import (
"github.com/pingcap/tidb/ast"
"github.com/pingcap/tidb/context"
"github.com/pingcap/tidb/expression"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/model"
"github.com/pingcap/tidb/mysql"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/util/sqlexec"
"github.com/pingcap/tidb/util/types"
Expand All @@ -45,7 +45,6 @@ type GrantExec struct {
Users []*ast.UserSpec

ctx context.Context
is infoschema.InfoSchema
done bool
}

Expand Down Expand Up @@ -495,7 +494,8 @@ func (e *GrantExec) getTargetSchema() (*model.DBInfo, error) {
}
//check if db exists
schema := model.NewCIStr(dbName)
db, ok := e.is.SchemaByName(schema)
is := sessionctx.GetDomain(e.ctx).InfoSchema()
db, ok := is.SchemaByName(schema)
if !ok {
return nil, errors.Errorf("Unknown schema name: %s", dbName)
}
Expand All @@ -509,7 +509,8 @@ func (e *GrantExec) getTargetSchemaAndTable() (*model.DBInfo, table.Table, error
return nil, nil, errors.Trace(err)
}
name := model.NewCIStr(e.Level.TableName)
tbl, err := e.is.TableByName(db.Name, name)
is := sessionctx.GetDomain(e.ctx).InfoSchema()
tbl, err := is.TableByName(db.Name, name)
if err != nil {
return nil, nil, errors.Trace(err)
}
Expand Down
3 changes: 2 additions & 1 deletion executor/prepared.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/parser"
"github.com/pingcap/tidb/plan"
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/util/sqlexec"
)

Expand Down Expand Up @@ -282,7 +283,7 @@ func CompileExecutePreparedStmt(ctx context.Context, ID uint32, args ...interfac
execPlan.UsingVars[i] = &expression.Constant{Value: value.Datum, RetType: &value.Type}
}
sa := &statement{
is: GetInfoSchema(ctx),
is: sessionctx.GetDomain(ctx).InfoSchema(),
plan: execPlan,
}
return sa
Expand Down
13 changes: 11 additions & 2 deletions executor/union_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -72,12 +72,21 @@ type dirtyTable struct {
truncated bool
}

type dirtyDBKeyType int

func (u dirtyDBKeyType) String() string {
return "dirtyDBKeyType"
}

// DirtyDBKey is the key to *dirtyDB for a context.
const DirtyDBKey dirtyDBKeyType = 1

func getDirtyDB(ctx context.Context) *dirtyDB {
var udb *dirtyDB
x := ctx.GetSessionVars().TxnCtx.DirtyDB
x := ctx.Value(DirtyDBKey)
if x == nil {
udb = &dirtyDB{tables: make(map[int64]*dirtyTable)}
ctx.GetSessionVars().TxnCtx.DirtyDB = udb
ctx.SetValue(DirtyDBKey, udb)
} else {
udb = x.(*dirtyDB)
}
Expand Down
2 changes: 2 additions & 0 deletions kv/kv.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,8 @@ const (
// PresumeKeyNotExistsError is the option key for error.
// When PresumeKeyNotExists is set and condition is not match, should throw the error.
PresumeKeyNotExistsError
// RetryAttempts is the number of txn retry attempt.
RetryAttempts
// BinlogData is the binlog data to write.
BinlogData
)
Expand Down
Loading

0 comments on commit d406c77

Please sign in to comment.