diff --git a/cmd/explaintest/r/explain-non-select-stmt.result b/cmd/explaintest/r/explain-non-select-stmt.result index 8a323cfb284c2..98b9dd697c76f 100644 --- a/cmd/explaintest/r/explain-non-select-stmt.result +++ b/cmd/explaintest/r/explain-non-select-stmt.result @@ -11,16 +11,18 @@ Insert_1 N/A root N/A └─TableFullScan_6 10000.00 cop[tikv] table:t, keep order:false, stats:pseudo explain delete from t where a > 100; id estRows task operator info -Delete_4 N/A root N/A -└─TableReader_8 3333.33 root data:Selection_7 - └─Selection_7 3333.33 cop[tikv] gt(test.t.a, 100) - └─TableFullScan_6 10000.00 cop[tikv] table:t, keep order:false, stats:pseudo +Delete_5 N/A root N/A +└─SelectLock_7 3333.33 root for update + └─TableReader_10 3333.33 root data:Selection_9 + └─Selection_9 3333.33 cop[tikv] gt(test.t.a, 100) + └─TableFullScan_8 10000.00 cop[tikv] table:t, keep order:false, stats:pseudo explain update t set b = 100 where a = 200; id estRows task operator info -Update_4 N/A root N/A -└─TableReader_8 10.00 root data:Selection_7 - └─Selection_7 10.00 cop[tikv] eq(test.t.a, 200) - └─TableFullScan_6 10000.00 cop[tikv] table:t, keep order:false, stats:pseudo +Update_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─TableReader_10 10.00 root data:Selection_9 + └─Selection_9 10.00 cop[tikv] eq(test.t.a, 200) + └─TableFullScan_8 10000.00 cop[tikv] table:t, keep order:false, stats:pseudo explain replace into t select a, 100 from t; id estRows task operator info Insert_1 N/A root N/A diff --git a/cmd/explaintest/r/explain_easy.result b/cmd/explaintest/r/explain_easy.result index c02c0878d5dd3..34f9dded43276 100644 --- a/cmd/explaintest/r/explain_easy.result +++ b/cmd/explaintest/r/explain_easy.result @@ -57,10 +57,11 @@ Update_2 N/A root N/A └─Point_Get_1 1.00 root table:t1, handle:1 explain delete from t1 where t1.c2 = 1; id estRows task operator info -Delete_4 N/A root N/A -└─IndexLookUp_11 10.00 root - ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t1, index:c2, range:[1,1], keep order:false, stats:pseudo - └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t1, keep order:false, stats:pseudo +Delete_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─IndexLookUp_13 10.00 root + ├─IndexRangeScan_11(Build) 10.00 cop[tikv] table:t1, index:c2, range:[1,1], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 cop[tikv] table:t1, keep order:false, stats:pseudo explain select count(b.c2) from t1 a, t2 b where a.c1 = b.c2 group by a.c1; id estRows task operator info Projection_11 9990.00 root cast(Column#8, bigint(21) BINARY)->Column#7 diff --git a/cmd/explaintest/r/explain_easy_stats.result b/cmd/explaintest/r/explain_easy_stats.result index d78928658424c..e9b083f3ec8f2 100644 --- a/cmd/explaintest/r/explain_easy_stats.result +++ b/cmd/explaintest/r/explain_easy_stats.result @@ -58,10 +58,11 @@ Update_2 N/A root N/A └─Point_Get_1 1.00 root table:t1, handle:1 explain delete from t1 where t1.c2 = 1; id estRows task operator info -Delete_4 N/A root N/A -└─IndexLookUp_11 0.00 root - ├─IndexRangeScan_9(Build) 0.00 cop[tikv] table:t1, index:c2, range:[1,1], keep order:false - └─TableRowIDScan_10(Probe) 0.00 cop[tikv] table:t1, keep order:false +Delete_5 N/A root N/A +└─SelectLock_7 0.00 root for update + └─IndexLookUp_13 0.00 root + ├─IndexRangeScan_11(Build) 0.00 cop[tikv] table:t1, index:c2, range:[1,1], keep order:false + └─TableRowIDScan_12(Probe) 0.00 cop[tikv] table:t1, keep order:false explain select count(b.c2) from t1 a, t2 b where a.c1 = b.c2 group by a.c1; id estRows task operator info Projection_11 1985.00 root cast(Column#8, bigint(21) BINARY)->Column#7 diff --git a/cmd/explaintest/r/explain_generate_column_substitute.result b/cmd/explaintest/r/explain_generate_column_substitute.result index a485bd6215767..b4d26bb35c954 100644 --- a/cmd/explaintest/r/explain_generate_column_substitute.result +++ b/cmd/explaintest/r/explain_generate_column_substitute.result @@ -135,28 +135,32 @@ b+a 8 desc update t set a=1 where a+1 = 3; id estRows task operator info -Update_4 N/A root N/A -└─IndexLookUp_11 10.00 root - ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:c, range:[3,3], keep order:false, stats:pseudo - └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo +Update_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─IndexLookUp_13 10.00 root + ├─IndexRangeScan_11(Build) 10.00 cop[tikv] table:t, index:c, range:[3,3], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo desc update t set a=2, b = 3 where b+a = 3; id estRows task operator info -Update_4 N/A root N/A -└─IndexLookUp_11 10.00 root - ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:e, range:[3,3], keep order:false, stats:pseudo - └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo +Update_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─IndexLookUp_13 10.00 root + ├─IndexRangeScan_11(Build) 10.00 cop[tikv] table:t, index:e, range:[3,3], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo desc delete from t where a+1 = 3; id estRows task operator info -Delete_4 N/A root N/A -└─IndexLookUp_11 10.00 root - ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:c, range:[3,3], keep order:false, stats:pseudo - └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo +Delete_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─IndexLookUp_13 10.00 root + ├─IndexRangeScan_11(Build) 10.00 cop[tikv] table:t, index:c, range:[3,3], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo desc delete from t where b+a = 0; id estRows task operator info -Delete_4 N/A root N/A -└─IndexLookUp_11 10.00 root - ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:e, range:[0,0], keep order:false, stats:pseudo - └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo +Delete_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─IndexLookUp_13 10.00 root + ├─IndexRangeScan_11(Build) 10.00 cop[tikv] table:t, index:e, range:[0,0], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo alter table t drop index idx_c; alter table t drop index idx_e; alter table t add index expr_idx_c((a+1)); @@ -302,25 +306,29 @@ b+a 8 desc update t set a=1 where a+1 = 3; id estRows task operator info -Update_4 N/A root N/A -└─IndexLookUp_11 10.00 root - ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:_V$_expr_idx_c_0, range:[3,3], keep order:false, stats:pseudo - └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo +Update_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─IndexLookUp_13 10.00 root + ├─IndexRangeScan_11(Build) 10.00 cop[tikv] table:t, index:_V$_expr_idx_c_0, range:[3,3], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo desc update t set a=2, b = 3 where b+a = 3; id estRows task operator info -Update_4 N/A root N/A -└─IndexLookUp_11 10.00 root - ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:_V$_expr_idx_e_0, range:[3,3], keep order:false, stats:pseudo - └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo +Update_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─IndexLookUp_13 10.00 root + ├─IndexRangeScan_11(Build) 10.00 cop[tikv] table:t, index:_V$_expr_idx_e_0, range:[3,3], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo desc delete from t where a+1 = 3; id estRows task operator info -Delete_4 N/A root N/A -└─IndexLookUp_11 10.00 root - ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:_V$_expr_idx_c_0, range:[3,3], keep order:false, stats:pseudo - └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo +Delete_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─IndexLookUp_13 10.00 root + ├─IndexRangeScan_11(Build) 10.00 cop[tikv] table:t, index:_V$_expr_idx_c_0, range:[3,3], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo desc delete from t where b+a = 0; id estRows task operator info -Delete_4 N/A root N/A -└─IndexLookUp_11 10.00 root - ├─IndexRangeScan_9(Build) 10.00 cop[tikv] table:t, index:_V$_expr_idx_e_0, range:[0,0], keep order:false, stats:pseudo - └─TableRowIDScan_10(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo +Delete_5 N/A root N/A +└─SelectLock_7 10.00 root for update + └─IndexLookUp_13 10.00 root + ├─IndexRangeScan_11(Build) 10.00 cop[tikv] table:t, index:_V$_expr_idx_e_0, range:[0,0], keep order:false, stats:pseudo + └─TableRowIDScan_12(Probe) 10.00 cop[tikv] table:t, keep order:false, stats:pseudo diff --git a/executor/adapter.go b/executor/adapter.go index 53001d4f1aebc..184c7f04723c4 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -682,6 +682,7 @@ type pessimisticTxn interface { // buildExecutor build a executor from plan, prepared statement may need additional procedure. func (a *ExecStmt) buildExecutor() (Executor, error) { ctx := a.Ctx + stmtCtx := ctx.GetSessionVars().StmtCtx if _, ok := a.Plan.(*plannercore.Execute); !ok { // Do not sync transaction for Execute statement, because the real optimization work is done in // "ExecuteExec.Build". @@ -701,7 +702,6 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { return nil, err } - stmtCtx := ctx.GetSessionVars().StmtCtx if stmtPri := stmtCtx.Priority; stmtPri == mysql.NoPriority { switch { case useMaxTS: @@ -735,7 +735,7 @@ func (a *ExecStmt) buildExecutor() (Executor, error) { } e = executorExec.stmtExec } - a.isSelectForUpdate = b.isSelectForUpdate + a.isSelectForUpdate = b.hasLock && (!stmtCtx.InDeleteStmt && !stmtCtx.InUpdateStmt) return e, nil } diff --git a/executor/builder.go b/executor/builder.go index e750498cdd27e..f2e5e436f5b37 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -77,8 +77,8 @@ type executorBuilder struct { is infoschema.InfoSchema startTS uint64 // cached when the first time getStartTS() is called // err is set when there is error happened during Executor building process. - err error - isSelectForUpdate bool + err error + hasLock bool } func newExecutorBuilder(ctx sessionctx.Context, is infoschema.InfoSchema) *executorBuilder { @@ -560,7 +560,7 @@ func (b *executorBuilder) buildDeallocate(v *plannercore.Deallocate) Executor { } func (b *executorBuilder) buildSelectLock(v *plannercore.PhysicalLock) Executor { - b.isSelectForUpdate = true + b.hasLock = true if b.err = b.updateForUpdateTSIfNeeded(v.Children()[0]); b.err != nil { return nil } @@ -2930,7 +2930,7 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan waitTime: plan.LockWaitTime, } if e.lock { - b.isSelectForUpdate = e.lock + b.hasLock = true } var capacity int if plan.IndexInfo != nil { diff --git a/executor/executor.go b/executor/executor.go index 9f6ed4853495c..91f4ef7f9c79d 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -897,7 +897,10 @@ func newLockCtx(seVars *variable.SessionVars, lockWaitTime int64) *kv.LockCtx { // locked by others. used for (select for update nowait) situation // except 0 means alwaysWait 1 means nowait func doLockKeys(ctx context.Context, se sessionctx.Context, lockCtx *kv.LockCtx, keys ...kv.Key) error { - se.GetSessionVars().TxnCtx.ForUpdate = true + sctx := se.GetSessionVars().StmtCtx + if !sctx.InUpdateStmt && !sctx.InDeleteStmt { + se.GetSessionVars().TxnCtx.ForUpdate = true + } // Lock keys only once when finished fetching all results. txn, err := se.Txn(true) if err != nil { diff --git a/executor/point_get.go b/executor/point_get.go index b1bfc5cb0e9e1..32452deff5808 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -47,7 +47,9 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) Executor { } e.base().initCap = 1 e.base().maxChunkSize = 1 - b.isSelectForUpdate = p.IsForUpdate + if p.Lock { + b.hasLock = true + } e.Init(p, startTS) return e } diff --git a/planner/core/expression_rewriter.go b/planner/core/expression_rewriter.go index e0efdfce50efa..98684ebd5195d 100644 --- a/planner/core/expression_rewriter.go +++ b/planner/core/expression_rewriter.go @@ -701,7 +701,7 @@ func (er *expressionRewriter) handleExistSubquery(ctx context.Context, v *ast.Ex } er.ctxStackAppend(er.p.Schema().Columns[er.p.Schema().Len()-1], er.p.OutputNames()[er.p.Schema().Len()-1]) } else { - physicalPlan, _, err := DoOptimize(ctx, er.b.optFlag, np) + physicalPlan, _, err := DoOptimize(ctx, er.sctx, er.b.optFlag, np) if err != nil { er.err = err return v, true @@ -864,7 +864,7 @@ func (er *expressionRewriter) handleScalarSubquery(ctx context.Context, v *ast.S } return v, true } - physicalPlan, _, err := DoOptimize(ctx, er.b.optFlag, np) + physicalPlan, _, err := DoOptimize(ctx, er.sctx, er.b.optFlag, np) if err != nil { er.err = err return v, true diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 3142c3bf72e0c..a289a90c44e3a 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -3163,6 +3163,11 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( return nil, err } } + if !b.ctx.GetSessionVars().IsAutocommit() || b.ctx.GetSessionVars().InTxn() { + if !update.MultipleTable { + p = b.buildSelectLock(p, ast.SelectLockForUpdate) + } + } if update.Order != nil { p, err = b.buildSort(ctx, p, update.Order.Items, nil, nil) @@ -3201,7 +3206,7 @@ func (b *PlanBuilder) buildUpdate(ctx context.Context, update *ast.UpdateStmt) ( updt.names = p.OutputNames() // We cannot apply projection elimination when building the subplan, because // columns in orderedList cannot be resolved. - updt.SelectPlan, _, err = DoOptimize(ctx, b.optFlag&^flagEliminateProjection, p) + updt.SelectPlan, _, err = DoOptimize(ctx, b.ctx, b.optFlag&^flagEliminateProjection, p) if err != nil { return nil, err } @@ -3398,6 +3403,11 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( return nil, err } } + if !b.ctx.GetSessionVars().IsAutocommit() || b.ctx.GetSessionVars().InTxn() { + if !delete.IsMultiTable { + p = b.buildSelectLock(p, ast.SelectLockForUpdate) + } + } if delete.Order != nil { p, err = b.buildSort(ctx, p, delete.Order.Items, nil, nil) @@ -3424,7 +3434,7 @@ func (b *PlanBuilder) buildDelete(ctx context.Context, delete *ast.DeleteStmt) ( }.Init(b.ctx) del.names = p.OutputNames() - del.SelectPlan, _, err = DoOptimize(ctx, b.optFlag, p) + del.SelectPlan, _, err = DoOptimize(ctx, b.ctx, b.optFlag, p) if err != nil { return nil, err } diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index f765d8e38009e..0c77b60a10031 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -119,7 +119,7 @@ func CheckTableLock(ctx sessionctx.Context, is infoschema.InfoSchema, vs []visit } // DoOptimize optimizes a logical plan to a physical plan. -func DoOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { +func DoOptimize(ctx context.Context, sctx sessionctx.Context, flag uint64, logic LogicalPlan) (PhysicalPlan, float64, error) { logic, err := logicalOptimize(ctx, flag, logic) if err != nil { return nil, 0, err @@ -131,13 +131,14 @@ func DoOptimize(ctx context.Context, flag uint64, logic LogicalPlan) (PhysicalPl if err != nil { return nil, 0, err } - finalPlan := postOptimize(physical) + finalPlan := postOptimize(sctx, physical) return finalPlan, cost, nil } -func postOptimize(plan PhysicalPlan) PhysicalPlan { +func postOptimize(sctx sessionctx.Context, plan PhysicalPlan) PhysicalPlan { plan = eliminatePhysicalProjection(plan) plan = injectExtraProjection(plan) + plan = eliminateUnionScanAndLock(sctx, plan) return plan } @@ -187,6 +188,74 @@ func physicalOptimize(logic LogicalPlan) (PhysicalPlan, float64, error) { return t.plan(), t.cost(), err } +// eliminateUnionScanAndLock set lock property for PointGet and BatchPointGet and eliminates UnionScan and Lock. +func eliminateUnionScanAndLock(sctx sessionctx.Context, p PhysicalPlan) PhysicalPlan { + var pointGet *PointGetPlan + var batchPointGet *BatchPointGetPlan + var physLock *PhysicalLock + var unionScan *PhysicalUnionScan + iteratePhysicalPlan(p, func(p PhysicalPlan) bool { + if len(p.Children()) > 1 { + return false + } + switch x := p.(type) { + case *PointGetPlan: + pointGet = x + case *BatchPointGetPlan: + batchPointGet = x + case *PhysicalLock: + physLock = x + case *PhysicalUnionScan: + unionScan = x + } + return true + }) + if pointGet == nil && batchPointGet == nil { + return p + } + if physLock == nil && unionScan == nil { + return p + } + if physLock != nil { + lock, waitTime := getLockWaitTime(sctx, physLock.Lock) + if !lock { + return p + } + if pointGet != nil { + pointGet.Lock = lock + pointGet.LockWaitTime = waitTime + } else { + batchPointGet.Lock = lock + batchPointGet.LockWaitTime = waitTime + } + } + return transformPhysicalPlan(p, func(p PhysicalPlan) PhysicalPlan { + if p == physLock { + return p.Children()[0] + } + if p == unionScan { + return p.Children()[0] + } + return p + }) +} + +func iteratePhysicalPlan(p PhysicalPlan, f func(p PhysicalPlan) bool) { + if !f(p) { + return + } + for _, child := range p.Children() { + iteratePhysicalPlan(child, f) + } +} + +func transformPhysicalPlan(p PhysicalPlan, f func(p PhysicalPlan) PhysicalPlan) PhysicalPlan { + for i, child := range p.Children() { + p.Children()[i] = transformPhysicalPlan(child, f) + } + return f(p) +} + func existsCartesianProduct(p LogicalPlan) bool { if join, ok := p.(*LogicalJoin); ok && len(join.EqualConditions) == 0 { return join.JoinType == InnerJoin || join.JoinType == LeftOuterJoin || join.JoinType == RightOuterJoin diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index baf0acd4f865d..2e837f63e77f8 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2202,7 +2202,7 @@ func (b *PlanBuilder) buildSelectPlanOfInsert(ctx context.Context, insert *ast.I } names := selectPlan.OutputNames() - insertPlan.SelectPlan, _, err = DoOptimize(ctx, b.optFlag, selectPlan.(LogicalPlan)) + insertPlan.SelectPlan, _, err = DoOptimize(ctx, b.ctx, b.optFlag, selectPlan.(LogicalPlan)) if err != nil { return err } diff --git a/planner/optimize.go b/planner/optimize.go index 29a5679e62a16..1aac017bb5d64 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -169,7 +169,7 @@ func optimize(ctx context.Context, sctx sessionctx.Context, node ast.Node, is in finalPlan, cost, err := cascades.DefaultOptimizer.FindBestPlan(sctx, logic) return finalPlan, names, cost, err } - finalPlan, cost, err := plannercore.DoOptimize(ctx, builder.GetOptFlag(), logic) + finalPlan, cost, err := plannercore.DoOptimize(ctx, sctx, builder.GetOptFlag(), logic) return finalPlan, names, cost, err } diff --git a/session/pessimistic_test.go b/session/pessimistic_test.go index feebceef0b689..18f11b43f6ebb 100644 --- a/session/pessimistic_test.go +++ b/session/pessimistic_test.go @@ -15,6 +15,7 @@ package session_test import ( "fmt" + "strings" "sync" "sync/atomic" "time" @@ -927,6 +928,46 @@ func (s *testPessimisticSuite) TestPessimisticReadCommitted(c *C) { tk.MustExec("commit;") } +func (s *testPessimisticSuite) TestPessimisticLockNonExistsKey(c *C) { + tk := testkit.NewTestKitWithInit(c, s.store) + tk1 := testkit.NewTestKitWithInit(c, s.store) + tk.MustExec("drop table if exists t") + tk.MustExec("create table t (k int primary key, c int)") + tk.MustExec("insert t values (1, 1), (3, 3), (5, 5)") + + // verify that select with project and filter on a non exists key still locks the key. + tk.MustExec("begin pessimistic") + tk.MustExec("insert t values (8, 8)") // Make the transaction dirty. + tk.MustQuery("select c + 1 from t where k = 2 and c = 2 for update").Check(testkit.Rows()) + explainStr := tk.MustQuery("explain select c + 1 from t where k = 2 and c = 2 for update").Rows()[0][0].(string) + c.Assert(strings.Contains(explainStr, "UnionScan"), IsFalse) + tk.MustQuery("select * from t where k in (4, 5, 7) for update").Check(testkit.Rows("5 5")) + + tk1.MustExec("begin pessimistic") + err := tk1.ExecToErr("select * from t where k = 2 for update nowait") + c.Check(tikv.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + err = tk1.ExecToErr("select * from t where k = 4 for update nowait") + c.Check(tikv.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + err = tk1.ExecToErr("select * from t where k = 7 for update nowait") + c.Check(tikv.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + tk.MustExec("rollback") + tk1.MustExec("rollback") + + // verify update and delete non exists keys still locks the key. + tk.MustExec("begin pessimistic") + tk.MustExec("insert t values (8, 8)") // Make the transaction dirty. + tk.MustExec("update t set c = c + 1 where k in (2, 3, 4) and c > 0") + tk.MustExec("delete from t where k in (5, 6, 7) and c > 0") + + tk1.MustExec("begin pessimistic") + err = tk1.ExecToErr("select * from t where k = 2 for update nowait") + c.Check(tikv.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + err = tk1.ExecToErr("select * from t where k = 6 for update nowait") + c.Check(tikv.ErrLockAcquireFailAndNoWaitSet.Equal(err), IsTrue) + tk.MustExec("rollback") + tk1.MustExec("rollback") +} + func (s *testPessimisticSuite) TestPessimisticCommitReadLock(c *C) { // set lock ttl to 3s, tk1 lock wait timeout is 2s atomic.StoreUint64(&tikv.ManagedLockTTL, 3000) diff --git a/session/session.go b/session/session.go index da689dce083ff..a194a7d616ec8 100644 --- a/session/session.go +++ b/session/session.go @@ -1323,7 +1323,7 @@ func (s *session) DropPreparedStmt(stmtID uint32) error { func (s *session) Txn(active bool) (kv.Transaction, error) { if !s.txn.validOrPending() && active { - return &s.txn, kv.ErrInvalidTxn + return &s.txn, errors.AddStack(kv.ErrInvalidTxn) } if s.txn.pending() && active { // Transaction is lazy initialized.