Skip to content

Commit

Permalink
*: not send tso request when point get with max tso (#11981) (#16336)
Browse files Browse the repository at this point in the history
  • Loading branch information
sre-bot authored Apr 14, 2020
1 parent f6b4a7c commit 489e313
Show file tree
Hide file tree
Showing 5 changed files with 36 additions and 3 deletions.
2 changes: 1 addition & 1 deletion executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -855,7 +855,7 @@ func (a *ExecStmt) SummaryStmt() {
// IsPointGetWithPKOrUniqueKeyByAutoCommit returns true when meets following conditions:
// 1. ctx is auto commit tagged
// 2. txn is not valid
// 2. plan is point get by pk, or point get by unique index (no double read)
// 3. plan is point get by pk, or point get by unique index (no double read)
func IsPointGetWithPKOrUniqueKeyByAutoCommit(ctx sessionctx.Context, p plannercore.Plan) (bool, error) {
// check auto commit
if !ctx.GetSessionVars().IsAutocommit() {
Expand Down
18 changes: 18 additions & 0 deletions planner/optimize.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ package planner

import (
"context"

"github.com/pingcap/parser/ast"
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
Expand All @@ -30,10 +31,15 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
if _, containTiKV := sctx.GetSessionVars().GetIsolationReadEngines()[kv.TiKV]; containTiKV {
fp := plannercore.TryFastPlan(sctx, node)
if fp != nil {
if !isPointGetWithoutDoubleRead(sctx, fp) {
sctx.PrepareTxnFuture(ctx)
}
return fp, nil
}
}

sctx.PrepareTxnFuture(ctx)

// build logical plan
sctx.GetSessionVars().PlanID = 0
sctx.GetSessionVars().PlanColumnID = 0
Expand Down Expand Up @@ -79,6 +85,18 @@ func Optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in
return plannercore.DoOptimize(ctx, builder.GetOptFlag(), logic)
}

// isPointGetWithoutDoubleRead returns true when meets following conditions:
// 1. ctx is auto commit tagged.
// 2. plan is point get by pk.
func isPointGetWithoutDoubleRead(ctx sessionctx.Context, p plannercore.Plan) bool {
if !ctx.GetSessionVars().IsAutocommit() {
return false
}

v, ok := p.(*plannercore.PointGetPlan)
return ok && v.IndexInfo == nil
}

func init() {
plannercore.OptimizeAstNode = Optimize
}
13 changes: 11 additions & 2 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -1239,6 +1239,7 @@ func (s *session) PrepareStmt(sql string) (stmtID uint32, paramCount int, fields
// NewPrepareExec may need startTS to build the executor, for example prepare statement has subquery in int.
// So we have to call PrepareTxnCtx here.
s.PrepareTxnCtx(ctx)
s.PrepareTxnFuture(ctx)
prepareExec := executor.NewPrepareExec(s, executor.GetInfoSchema(s), sql)
err = prepareExec.Next(ctx, nil)
if err != nil {
Expand Down Expand Up @@ -1893,8 +1894,6 @@ func (s *session) PrepareTxnCtx(ctx context.Context) {
return
}

txnFuture := s.getTxnFuture(ctx)
s.txn.changeInvalidToPending(txnFuture)
is := domain.GetDomain(s).InfoSchema()
s.sessionVars.TxnCtx = &variable.TransactionContext{
InfoSchema: is,
Expand All @@ -1911,6 +1910,16 @@ func (s *session) PrepareTxnCtx(ctx context.Context) {
}
}

// PrepareTxnFuture uses to try to get txn future.
func (s *session) PrepareTxnFuture(ctx context.Context) {
if s.txn.validOrPending() {
return
}

txnFuture := s.getTxnFuture(ctx)
s.txn.changeInvalidToPending(txnFuture)
}

// RefreshTxnCtx implements context.RefreshTxnCtx interface.
func (s *session) RefreshTxnCtx(ctx context.Context) error {
if err := s.doCommit(ctx); err != nil {
Expand Down
2 changes: 2 additions & 0 deletions sessionctx/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,6 +100,8 @@ type Context interface {
ReleaseAllTableLocks()
// HasLockedTables uses to check whether this session locked any tables.
HasLockedTables() bool
// PrepareTxnFuture uses to prepare txn by future.
PrepareTxnFuture(ctx context.Context)
}

type basicCtxType int
Expand Down
4 changes: 4 additions & 0 deletions util/mock/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -251,6 +251,10 @@ func (c *Context) HasLockedTables() bool {
return false
}

// PrepareTxnFuture implements the sessionctx.Context interface.
func (c *Context) PrepareTxnFuture(ctx context.Context) {
}

// Close implements the sessionctx.Context interface.
func (c *Context) Close() {
}
Expand Down

0 comments on commit 489e313

Please sign in to comment.