From 98f0d76bc770f661087b8eafaa3cbcb5a72df5ac Mon Sep 17 00:00:00 2001 From: JmPotato Date: Tue, 25 May 2021 14:25:33 +0800 Subject: [PATCH] *: clean up the deserted syntax of START TRANSACTION READ ONLY (#24813) --- executor/builder.go | 10 +- executor/simple.go | 104 ++--------------- executor/stale_txn_test.go | 136 +++++----------------- go.mod | 2 +- go.sum | 4 +- kv/fault_injection_test.go | 2 +- planner/core/common_plans.go | 4 +- planner/core/planbuilder.go | 5 +- session/session.go | 85 +++++--------- session/txn.go | 2 +- sessionctx/context.go | 13 +-- sessionctx/variable/varsutil.go | 12 +- store/tikv/kv.go | 27 ----- store/tikv/tests/2pc_test.go | 4 +- store/tikv/tests/extract_start_ts_test.go | 108 ----------------- store/tikv/txn.go | 86 ++------------ util/mock/context.go | 12 +- 17 files changed, 108 insertions(+), 508 deletions(-) delete mode 100644 store/tikv/tests/extract_start_ts_test.go diff --git a/executor/builder.go b/executor/builder.go index bf1ad7e46c9f0..2f08e604851a7 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -706,11 +706,11 @@ func (b *executorBuilder) buildSimple(v *plannercore.Simple) Executor { base := newBaseExecutor(b.ctx, v.Schema(), v.ID()) base.initCap = chunk.ZeroCapacity e := &SimpleExec{ - baseExecutor: base, - Statement: v.Statement, - IsFromRemote: v.IsFromRemote, - is: b.is, - StalenessTxnOption: v.StalenessTxnOption, + baseExecutor: base, + Statement: v.Statement, + IsFromRemote: v.IsFromRemote, + is: b.is, + staleTxnStartTS: v.StaleTxnStartTS, } return e } diff --git a/executor/simple.go b/executor/simple.go index 9a0b4b919b1e3..2e9f8c89ea4be 100644 --- a/executor/simple.go +++ b/executor/simple.go @@ -39,10 +39,7 @@ import ( "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/variable" - "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" - "github.com/pingcap/tidb/types" - driver "github.com/pingcap/tidb/types/parser_driver" "github.com/pingcap/tidb/util" "github.com/pingcap/tidb/util/chunk" "github.com/pingcap/tidb/util/collate" @@ -75,8 +72,8 @@ type SimpleExec struct { done bool is infoschema.InfoSchema - // StalenessTxnOption is used to execute the staleness txn during a read-only begin statement. - StalenessTxnOption *sessionctx.StalenessTxnOption + // staleTxnStartTS is the StartTS that is used to execute the staleness txn during a read-only begin statement. + staleTxnStartTS uint64 } func (e *baseExecutor) getSysSession() (sessionctx.Context, error) { @@ -573,14 +570,18 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { // always create a new Txn instead of reusing it. if s.ReadOnly { enableNoopFuncs := e.ctx.GetSessionVars().EnableNoopFuncs - if !enableNoopFuncs && s.AsOf == nil && s.Bound == nil { + if !enableNoopFuncs && s.AsOf == nil { return expression.ErrFunctionsNoopImpl.GenWithStackByArgs("READ ONLY") } if s.AsOf != nil { - return e.executeStartTransactionReadOnlyWithBoundedStaleness(ctx, s) - } - if s.Bound != nil { - return e.executeStartTransactionReadOnlyWithTimestampBound(ctx, s) + if err := e.ctx.NewTxnWithStartTS(ctx, e.staleTxnStartTS); err != nil { + return err + } + // With START TRANSACTION, autocommit remains disabled until you end + // the transaction with COMMIT or ROLLBACK. The autocommit mode then + // reverts to its previous state. + e.ctx.GetSessionVars().SetInTxn(true) + return nil } } @@ -620,89 +621,6 @@ func (e *SimpleExec) executeBegin(ctx context.Context, s *ast.BeginStmt) error { return nil } -func (e *SimpleExec) executeStartTransactionReadOnlyWithBoundedStaleness(ctx context.Context, s *ast.BeginStmt) error { - if e.StalenessTxnOption == nil { - return errors.New("Failed to get timestamp during start transaction read only as of timestamp") - } - if err := e.ctx.NewTxnWithStalenessOption(ctx, *e.StalenessTxnOption); err != nil { - return err - } - - // With START TRANSACTION, autocommit remains disabled until you end - // the transaction with COMMIT or ROLLBACK. The autocommit mode then - // reverts to its previous state. - e.ctx.GetSessionVars().SetInTxn(true) - return nil -} - -// TODO: deprecate this syntax and only keep `AS OF TIMESTAMP` statement. -func (e *SimpleExec) executeStartTransactionReadOnlyWithTimestampBound(ctx context.Context, s *ast.BeginStmt) error { - opt := sessionctx.StalenessTxnOption{} - opt.Mode = s.Bound.Mode - switch s.Bound.Mode { - case ast.TimestampBoundReadTimestamp: - // TODO: support funcCallExpr in future - v, ok := s.Bound.Timestamp.(*driver.ValueExpr) - if !ok { - return errors.New("Invalid value for Bound Timestamp") - } - t, err := types.ParseTime(e.ctx.GetSessionVars().StmtCtx, v.GetString(), v.GetType().Tp, types.GetFsp(v.GetString())) - if err != nil { - return err - } - gt, err := t.GoTime(e.ctx.GetSessionVars().TimeZone) - if err != nil { - return err - } - opt.StartTS = oracle.GoTimeToTS(gt) - case ast.TimestampBoundExactStaleness: - // TODO: support funcCallExpr in future - v, ok := s.Bound.Timestamp.(*driver.ValueExpr) - if !ok { - return errors.New("Invalid value for Bound Timestamp") - } - d, err := types.ParseDuration(e.ctx.GetSessionVars().StmtCtx, v.GetString(), types.GetFsp(v.GetString())) - if err != nil { - return err - } - opt.PrevSec = uint64(d.Seconds()) - case ast.TimestampBoundMaxStaleness: - v, ok := s.Bound.Timestamp.(*driver.ValueExpr) - if !ok { - return errors.New("Invalid value for Bound Timestamp") - } - d, err := types.ParseDuration(e.ctx.GetSessionVars().StmtCtx, v.GetString(), types.GetFsp(v.GetString())) - if err != nil { - return err - } - opt.PrevSec = uint64(d.Seconds()) - case ast.TimestampBoundMinReadTimestamp: - v, ok := s.Bound.Timestamp.(*driver.ValueExpr) - if !ok { - return errors.New("Invalid value for Bound Timestamp") - } - t, err := types.ParseTime(e.ctx.GetSessionVars().StmtCtx, v.GetString(), v.GetType().Tp, types.GetFsp(v.GetString())) - if err != nil { - return err - } - gt, err := t.GoTime(e.ctx.GetSessionVars().TimeZone) - if err != nil { - return err - } - opt.StartTS = oracle.GoTimeToTS(gt) - } - err := e.ctx.NewTxnWithStalenessOption(ctx, opt) - if err != nil { - return err - } - - // With START TRANSACTION, autocommit remains disabled until you end - // the transaction with COMMIT or ROLLBACK. The autocommit mode then - // reverts to its previous state. - e.ctx.GetSessionVars().SetInTxn(true) - return nil -} - func (e *SimpleExec) executeRevokeRole(s *ast.RevokeRoleStmt) error { for _, role := range s.Roles { exists, err := userExists(e.ctx, role.Username, role.Hostname) diff --git a/executor/stale_txn_test.go b/executor/stale_txn_test.go index a9336fbfac882..1f4f24a7f9f37 100644 --- a/executor/stale_txn_test.go +++ b/executor/stale_txn_test.go @@ -20,7 +20,6 @@ import ( . "github.com/pingcap/check" "github.com/pingcap/failpoint" "github.com/pingcap/tidb/ddl/placement" - "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/util/testkit" ) @@ -32,66 +31,38 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { sql string IsStaleness bool expectPhysicalTS int64 - preSec int64 txnScope string zone string }{ { - name: "TimestampBoundExactStaleness", - preSQL: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:20';`, - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, - IsStaleness: true, - expectPhysicalTS: 1599321600000, - txnScope: "local", - zone: "sh", - }, - { - name: "TimestampBoundReadTimestamp", + name: "AsOfTimestamp", preSQL: "begin", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, IsStaleness: true, expectPhysicalTS: 1599321600000, txnScope: "local", - zone: "bj", - }, - { - name: "TimestampBoundExactStaleness", - preSQL: "begin", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:20';`, - IsStaleness: true, - preSec: 20, - txnScope: "local", - zone: "sh", - }, - { - name: "TimestampBoundExactStaleness", - preSQL: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:20';`, - IsStaleness: true, - preSec: 20, - txnScope: "local", - zone: "sz", + zone: "sh", }, { - name: "begin after TimestampBoundReadTimestamp", - preSQL: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`, + name: "begin after AsOfTimestamp", + preSQL: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, sql: "begin", IsStaleness: false, - txnScope: kv.GlobalTxnScope, + txnScope: oracle.GlobalTxnScope, zone: "", }, { - name: "AsOfTimestamp", + name: "AsOfTimestamp with tidb_bounded_staleness", preSQL: "begin", - sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP tidb_bounded_staleness('2015-09-21 00:07:01', NOW());`, IsStaleness: true, - expectPhysicalTS: 1599321600000, + expectPhysicalTS: 1442765221000, txnScope: "local", - zone: "sh", + zone: "bj", }, { - name: "begin after AsOfTimestamp", - preSQL: `START TRANSACTION READ ONLY AS OF TIMESTAMP '2020-09-06 00:00:00';`, + name: "begin after AsOfTimestamp with tidb_bounded_staleness", + preSQL: `START TRANSACTION READ ONLY AS OF TIMESTAMP tidb_bounded_staleness('2015-09-21 00:07:01', NOW());`, sql: "begin", IsStaleness: false, txnScope: oracle.GlobalTxnScope, @@ -107,21 +78,15 @@ func (s *testStaleTxnSerialSuite) TestExactStalenessTransaction(c *C) { tk.MustExec(fmt.Sprintf("set @@txn_scope=%v", testcase.txnScope)) tk.MustExec(testcase.preSQL) tk.MustExec(testcase.sql) + c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.IsStaleness) if testcase.expectPhysicalTS > 0 { c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, testcase.expectPhysicalTS) - } else if testcase.preSec > 0 { - curSec := time.Now().Unix() - startTS := oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS) - // exact stale txn tolerate 2 seconds deviation for startTS - c.Assert(startTS, Greater, (curSec-testcase.preSec-2)*1000) - c.Assert(startTS, Less, (curSec-testcase.preSec+2)*1000) } else if !testcase.IsStaleness { - curSec := time.Now().Unix() + curTS := oracle.ExtractPhysical(oracle.GoTimeToTS(time.Now())) startTS := oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS) - c.Assert(curSec*1000-startTS, Less, time.Second/time.Millisecond) - c.Assert(startTS-curSec*1000, Less, time.Second/time.Millisecond) + c.Assert(curTS-startTS, Less, time.Second.Milliseconds()) + c.Assert(startTS-curTS, Less, time.Second.Milliseconds()) } - c.Assert(tk.Se.GetSessionVars().TxnCtx.IsStaleness, Equals, testcase.IsStaleness) tk.MustExec("commit") } failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") @@ -168,9 +133,6 @@ func (s *testStaleTxnSerialSuite) TestStaleReadKVRequest(c *C) { tk.MustExec(`START TRANSACTION READ ONLY AS OF TIMESTAMP NOW(3);`) tk.MustQuery(testcase.sql) tk.MustExec(`commit`) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND EXACT STALENESS '00:00:00';`) - tk.MustQuery(testcase.sql) - tk.MustExec(`commit`) } failpoint.Disable("github.com/pingcap/tidb/config/injectTxnScope") failpoint.Disable("github.com/pingcap/tidb/store/tikv/assertStoreLabels") @@ -199,16 +161,6 @@ func (s *testStaleTxnSerialSuite) TestStalenessAndHistoryRead(c *C) { tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) tk.MustExec("commit") - // set @@tidb_snapshot before staleness txn - tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`) - c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) - tk.MustExec("commit") - // set @@tidb_snapshot during staleness txn - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '2020-09-06 00:00:00';`) - tk.MustExec(`set @@tidb_snapshot="2016-10-08 16:45:26";`) - c.Assert(oracle.ExtractPhysical(tk.Se.GetSessionVars().TxnCtx.StartTS), Equals, int64(1599321600000)) - tk.MustExec("commit") } func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { @@ -217,7 +169,6 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") defer tk.MustExec(`drop table if exists t`) - tk.MustExec(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`) testcases := []struct { name string sql string @@ -225,36 +176,6 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { // compareWithSafeTS will be 0 if StartTS==SafeTS, -1 if StartTS < SafeTS, and +1 if StartTS > SafeTS. compareWithSafeTS int }{ - { - name: "max 20 seconds ago, safeTS 10 secs ago", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:20'`, - injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)), - compareWithSafeTS: 0, - }, - { - name: "max 10 seconds ago, safeTS 20 secs ago", - sql: `START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MAX STALENESS '00:00:10'`, - injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)), - compareWithSafeTS: 1, - }, - { - name: "max 20 seconds ago, safeTS 10 secs ago", - sql: func() string { - return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, - time.Now().Add(-20*time.Second).Format("2006-01-02 15:04:05")) - }(), - injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)), - compareWithSafeTS: 0, - }, - { - name: "max 10 seconds ago, safeTS 20 secs ago", - sql: func() string { - return fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND MIN READ TIMESTAMP '%v'`, - time.Now().Add(-10*time.Second).Format("2006-01-02 15:04:05")) - }(), - injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-20 * time.Second)), - compareWithSafeTS: 1, - }, { name: "20 seconds ago to now, safeTS 10 secs ago", sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP tidb_bounded_staleness(NOW() - INTERVAL 20 SECOND, NOW())`, @@ -273,6 +194,18 @@ func (s *testStaleTxnSerialSuite) TestTimeBoundedStalenessTxn(c *C) { injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-5 * time.Second)), compareWithSafeTS: -1, }, + { + name: "exact timestamp 5 seconds ago, safeTS 10 secs ago", + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP NOW() - INTERVAL 5 SECOND`, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-10 * time.Second)), + compareWithSafeTS: 1, + }, + { + name: "exact timestamp 10 seconds ago, safeTS 5 secs ago", + sql: `START TRANSACTION READ ONLY AS OF TIMESTAMP NOW() - INTERVAL 10 SECOND`, + injectSafeTS: oracle.GoTimeToTS(time.Now().Add(-5 * time.Second)), + compareWithSafeTS: -1, + }, } for _, testcase := range testcases { c.Log(testcase.name) @@ -300,21 +233,12 @@ func (s *testStaleTxnSerialSuite) TestStalenessTransactionSchemaVer(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t (id int primary key);") - // test exact + // test as of schemaVer1 := tk.Se.GetInfoSchema().SchemaMetaVersion() time1 := time.Now() tk.MustExec("drop table if exists t") c.Assert(schemaVer1, Less, tk.Se.GetInfoSchema().SchemaMetaVersion()) - tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY WITH TIMESTAMP BOUND READ TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) + tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time1.Format("2006-1-2 15:04:05.000"))) c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer1) tk.MustExec("commit") - - // test as of - schemaVer2 := tk.Se.GetInfoSchema().SchemaMetaVersion() - time2 := time.Now() - tk.MustExec("create table t (id int primary key);") - c.Assert(schemaVer2, Less, tk.Se.GetInfoSchema().SchemaMetaVersion()) - tk.MustExec(fmt.Sprintf(`START TRANSACTION READ ONLY AS OF TIMESTAMP '%s'`, time2.Format("2006-1-2 15:04:05.000"))) - c.Assert(tk.Se.GetInfoSchema().SchemaMetaVersion(), Equals, schemaVer2) - tk.MustExec("commit") } diff --git a/go.mod b/go.mod index 32d76211a667b..7b06ca2a8682d 100644 --- a/go.mod +++ b/go.mod @@ -37,7 +37,7 @@ require ( github.com/opentracing/opentracing-go v1.1.0 github.com/phayes/freeport v0.0.0-20180830031419-95f893ade6f2 github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 - github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible + github.com/pingcap/br v5.1.0-alpha.0.20210524083733-58e9e24fcb8e+incompatible github.com/pingcap/check v0.0.0-20200212061837-5e12011dc712 github.com/pingcap/errors v0.11.5-0.20201126102027-b0a155152ca3 github.com/pingcap/failpoint v0.0.0-20210316064728-7acb0f0a3dfd diff --git a/go.sum b/go.sum index c488db7c0347a..ac5cfe59f6b17 100644 --- a/go.sum +++ b/go.sum @@ -407,8 +407,8 @@ github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d h1:U+PMnTlV2tu7RuMK5e github.com/phf/go-queue v0.0.0-20170504031614-9abe38d0371d/go.mod h1:lXfE4PvvTW5xOjO6Mba8zDPyw8M93B6AQ7frTGnMlA8= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19 h1:IXpGy7y9HyoShAFmzW2OPF0xCA5EOoSTyZHwsgYk9Ro= github.com/pingcap/badger v1.5.1-0.20200908111422-2e78ee155d19/go.mod h1:LyrqUOHZrUDf9oGi1yoz1+qw9ckSIhQb5eMa1acOLNQ= -github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible h1:VF2oZgvBqSIMmplEWXGGmktuQGdGGIGWwptmjJFhQbU= -github.com/pingcap/br v5.0.0-nightly.0.20210419090151-03762465b589+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= +github.com/pingcap/br v5.1.0-alpha.0.20210524083733-58e9e24fcb8e+incompatible h1:rRJG+3sGr4rbC3PUGpIhaO4uVB3ZssyJxZyLW8yFwBE= +github.com/pingcap/br v5.1.0-alpha.0.20210524083733-58e9e24fcb8e+incompatible/go.mod h1:ymVmo50lQydxib0tmK5hHk4oteB7hZ0IMCArunwy3UQ= github.com/pingcap/check v0.0.0-20190102082844-67f458068fc8/go.mod h1:B1+S9LNcuMyLH/4HMTViQOJevkGiik3wW2AN9zb2fNQ= github.com/pingcap/check v0.0.0-20191107115940-caf2b9e6ccf4/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= github.com/pingcap/check v0.0.0-20191216031241-8a5a85928f12/go.mod h1:PYMCGwN0JHjoqGr3HrZoD+b8Tgx8bKnArhSq8YVzUMc= diff --git a/kv/fault_injection_test.go b/kv/fault_injection_test.go index c5e203151fe63..09ee3a14dae3c 100644 --- a/kv/fault_injection_test.go +++ b/kv/fault_injection_test.go @@ -35,7 +35,7 @@ func (s testFaultInjectionSuite) TestFaultInjectionBasic(c *C) { storage := NewInjectedStore(newMockStorage(), &cfg) txn, err := storage.Begin() c.Assert(err, IsNil) - _, err = storage.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(GlobalTxnScope).SetStartTs(0)) + _, err = storage.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(GlobalTxnScope).SetStartTS(0)) c.Assert(err, IsNil) ver := Version{Ver: 1} snap := storage.GetSnapshot(ver) diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index b42a84d926eeb..7170c991be051 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -724,8 +724,8 @@ type Simple struct { // Used for `global kill`. See https://github.com/pingcap/tidb/blob/master/docs/design/2020-06-01-global-kill.md. IsFromRemote bool - // StalenessTxnOption is the transaction option that will be built when planner builder calls buildSimple. - StalenessTxnOption *sessionctx.StalenessTxnOption + // StaleTxnStartTS is the StartTS that is used to build a staleness transaction by 'START TRANSACTION READ ONLY' statement. + StaleTxnStartTS uint64 } // PhysicalSimpleWrapper is a wrapper of `Simple` to implement physical plan interface. diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index f530908f8a6f7..cac9dab16d8ab 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -2340,10 +2340,7 @@ func (b *PlanBuilder) buildSimple(ctx context.Context, node ast.StmtNode) (Plan, if err != nil { return nil, err } - p.StalenessTxnOption = &sessionctx.StalenessTxnOption{ - Mode: ast.TimestampBoundReadTimestamp, - StartTS: startTS, - } + p.StaleTxnStartTS = startTS } } return p, nil diff --git a/session/session.go b/session/session.go index 9ef43bef3d7ed..a479ba956cf63 100644 --- a/session/session.go +++ b/session/session.go @@ -2000,6 +2000,35 @@ func (s *session) checkBeforeNewTxn(ctx context.Context) error { return nil } +// NewTxnWithStartTS create a transaction with the given StartTS. +func (s *session) NewTxnWithStartTS(ctx context.Context, startTS uint64) error { + if err := s.checkBeforeNewTxn(ctx); err != nil { + return err + } + txnScope := s.GetSessionVars().CheckAndGetTxnScope() + txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetStartTS(startTS)) + if err != nil { + return err + } + txn.SetVars(s.sessionVars.KVVars) + txn.SetOption(kv.IsStalenessReadOnly, true) + txn.SetOption(kv.TxnScope, txnScope) + s.txn.changeInvalidToValid(txn) + is, err := domain.GetDomain(s).GetSnapshotInfoSchema(txn.StartTS()) + if err != nil { + return errors.Trace(err) + } + s.sessionVars.TxnCtx = &variable.TransactionContext{ + InfoSchema: is, + CreateTime: time.Now(), + StartTS: txn.StartTS(), + ShardStep: int(s.sessionVars.ShardAllocateStep), + IsStaleness: true, + TxnScope: txnScope, + } + return nil +} + func (s *session) SetValue(key fmt.Stringer, value interface{}) { s.mu.Lock() s.mu.values[key] = value @@ -2648,7 +2677,7 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { } // no need to get txn from txnFutureCh since txn should init with startTs - txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTs(startTS)) + txn, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(s.GetSessionVars().CheckAndGetTxnScope()).SetStartTS(startTS)) if err != nil { return err } @@ -2661,60 +2690,6 @@ func (s *session) InitTxnWithStartTS(startTS uint64) error { return nil } -// NewTxnWithStalenessOption create a transaction with Staleness option -func (s *session) NewTxnWithStalenessOption(ctx context.Context, option sessionctx.StalenessTxnOption) error { - err := s.checkBeforeNewTxn(ctx) - if err != nil { - return err - } - var ( - txn kv.Transaction - txnScope = s.GetSessionVars().CheckAndGetTxnScope() - ) - switch option.Mode { - case ast.TimestampBoundReadTimestamp: - txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetStartTs(option.StartTS)) - if err != nil { - return err - } - case ast.TimestampBoundExactStaleness: - txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetPrevSec(option.PrevSec)) - if err != nil { - return err - } - case ast.TimestampBoundMaxStaleness: - txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetMaxPrevSec(option.PrevSec)) - if err != nil { - return err - } - case ast.TimestampBoundMinReadTimestamp: - txn, err = s.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(txnScope).SetMinStartTS(option.StartTS)) - if err != nil { - return err - } - default: - // For unsupported staleness txn cases, fallback to NewTxn - return s.NewTxn(ctx) - } - txn.SetVars(s.sessionVars.KVVars) - txn.SetOption(kv.IsStalenessReadOnly, true) - txn.SetOption(kv.TxnScope, txnScope) - s.txn.changeInvalidToValid(txn) - is, err := domain.GetDomain(s).GetSnapshotInfoSchema(txn.StartTS()) - if err != nil { - return errors.Trace(err) - } - s.sessionVars.TxnCtx = &variable.TransactionContext{ - InfoSchema: is, - CreateTime: time.Now(), - StartTS: txn.StartTS(), - ShardStep: int(s.sessionVars.ShardAllocateStep), - IsStaleness: true, - TxnScope: txnScope, - } - return nil -} - // GetStore gets the store of session. func (s *session) GetStore() kv.Storage { return s.store diff --git a/session/txn.go b/session/txn.go index 12b2bd7069db2..df4f2d7a62bed 100644 --- a/session/txn.go +++ b/session/txn.go @@ -436,7 +436,7 @@ type txnFuture struct { func (tf *txnFuture) wait() (kv.Transaction, error) { startTS, err := tf.future.Wait() if err == nil { - return tf.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(tf.txnScope).SetStartTs(startTS)) + return tf.store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(tf.txnScope).SetStartTS(startTS)) } else if config.GetGlobalConfig().Store == "unistore" { return nil, err } diff --git a/sessionctx/context.go b/sessionctx/context.go index 68767aef30e7c..96ede63f901ae 100644 --- a/sessionctx/context.go +++ b/sessionctx/context.go @@ -17,7 +17,6 @@ import ( "context" "fmt" - "github.com/pingcap/parser/ast" "github.com/pingcap/parser/model" "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/owner" @@ -42,6 +41,8 @@ type Context interface { // If old transaction is valid, it is committed first. // It's used in BEGIN statement and DDL statements to commit old transaction. NewTxn(context.Context) error + // NewTxnWithStartTS initializes a transaction with the given StartTS. + NewTxnWithStartTS(ctx context.Context, startTS uint64) error // Txn returns the current transaction which is created before executing a statement. // The returned kv.Transaction is not nil, but it maybe pending or invalid. @@ -83,9 +84,6 @@ type Context interface { // It should be called right before we builds an executor. InitTxnWithStartTS(startTS uint64) error - // NewTxnWithStalenessOption initializes a transaction with StalenessTxnOption. - NewTxnWithStalenessOption(ctx context.Context, option StalenessTxnOption) error - // GetStore returns the store of session. GetStore() kv.Storage @@ -151,10 +149,3 @@ const ( // LastExecuteDDL is the key for whether the session execute a ddl command last time. LastExecuteDDL basicCtxType = 3 ) - -// StalenessTxnOption represents available options for the InitTxnWithStaleness -type StalenessTxnOption struct { - Mode ast.TimestampBoundMode - PrevSec uint64 - StartTS uint64 -} diff --git a/sessionctx/variable/varsutil.go b/sessionctx/variable/varsutil.go index e47b5d46b9aa7..d42128e73c612 100644 --- a/sessionctx/variable/varsutil.go +++ b/sessionctx/variable/varsutil.go @@ -23,6 +23,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/parser/charset" "github.com/pingcap/parser/mysql" + "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/collate" "github.com/pingcap/tidb/util/timeutil" @@ -200,9 +201,6 @@ func GetGlobalSystemVar(s *SessionVars, name string) (string, error) { return sv.GetGlobalFromHook(s) } -// epochShiftBits is used to reserve logical part of the timestamp. -const epochShiftBits = 18 - // SetSessionSystemVar sets system variable and updates SessionVars states. func SetSessionSystemVar(vars *SessionVars, name string, value string) error { sysVar := GetSysVar(name) @@ -376,16 +374,10 @@ func setSnapshotTS(s *SessionVars, sVal string) error { } t1, err := t.GoTime(s.TimeZone) - s.SnapshotTS = GoTimeToTS(t1) + s.SnapshotTS = oracle.GoTimeToTS(t1) return err } -// GoTimeToTS converts a Go time to uint64 timestamp. -func GoTimeToTS(t time.Time) uint64 { - ts := (t.UnixNano() / int64(time.Millisecond)) << epochShiftBits - return uint64(ts) -} - // serverGlobalVariable is used to handle variables that acts in server and global scope. type serverGlobalVariable struct { sync.Mutex diff --git a/store/tikv/kv.go b/store/tikv/kv.go index 622313f382abd..e7fe6a8309475 100644 --- a/store/tikv/kv.go +++ b/store/tikv/kv.go @@ -240,13 +240,6 @@ func (s *KVStore) CurrentTimestamp(txnScope string) (uint64, error) { } func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, error) { - failpoint.Inject("MockCurrentTimestamp", func(val failpoint.Value) { - if v, ok := val.(int); ok { - failpoint.Return(uint64(v), nil) - } else { - panic("MockCurrentTimestamp should be a number, try use this failpoint with \"return(ts)\"") - } - }) if span := opentracing.SpanFromContext(bo.GetCtx()); span != nil && span.Tracer() != nil { span1 := span.Tracer().StartSpan("TiKVStore.getTimestampWithRetry", opentracing.ChildOf(span.Context())) defer span1.Finish() @@ -275,26 +268,6 @@ func (s *KVStore) getTimestampWithRetry(bo *Backoffer, txnScope string) (uint64, } } -func (s *KVStore) getStalenessTimestamp(bo *Backoffer, txnScope string, prevSec uint64) (uint64, error) { - failpoint.Inject("MockStalenessTimestamp", func(val failpoint.Value) { - if v, ok := val.(int); ok { - failpoint.Return(uint64(v), nil) - } else { - panic("MockStalenessTimestamp should be a number, try use this failpoint with \"return(ts)\"") - } - }) - for { - startTS, err := s.oracle.GetStaleTimestamp(bo.GetCtx(), txnScope, prevSec) - if err == nil { - return startTS, nil - } - err = bo.Backoff(retry.BoPDRPC, errors.Errorf("get staleness timestamp failed: %v", err)) - if err != nil { - return 0, errors.Trace(err) - } - } -} - func (s *KVStore) nextReplicaReadSeed() uint32 { return atomic.AddUint32(&s.replicaReadSeed, 1) } diff --git a/store/tikv/tests/2pc_test.go b/store/tikv/tests/2pc_test.go index 8fea337bfd61b..d56436232e935 100644 --- a/store/tikv/tests/2pc_test.go +++ b/store/tikv/tests/2pc_test.go @@ -602,12 +602,12 @@ func (s *testCommitterSuite) TestRejectCommitTS(c *C) { // Use max.Uint64 to read the data and success. // That means the final commitTS > startTS+2, it's not the one we provide. // So we cover the rety commitTS logic. - txn1, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTs(committer.GetStartTS() + 2)) + txn1, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTS(committer.GetStartTS() + 2)) c.Assert(err, IsNil) _, err = txn1.Get(bo.GetCtx(), []byte("x")) c.Assert(tikverr.IsErrNotFound(err), IsTrue) - txn2, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTs(math.MaxUint64)) + txn2, err := s.store.BeginWithOption(tikv.DefaultStartTSOption().SetStartTS(math.MaxUint64)) c.Assert(err, IsNil) val, err := txn2.Get(bo.GetCtx(), []byte("x")) c.Assert(err, IsNil) diff --git a/store/tikv/tests/extract_start_ts_test.go b/store/tikv/tests/extract_start_ts_test.go deleted file mode 100644 index 82f37796dce8a..0000000000000 --- a/store/tikv/tests/extract_start_ts_test.go +++ /dev/null @@ -1,108 +0,0 @@ -// Copyright 2021 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 tikv_test - -import ( - . "github.com/pingcap/check" - "github.com/pingcap/failpoint" - "github.com/pingcap/kvproto/pkg/metapb" - "github.com/pingcap/tidb/store/mockstore/unistore" - "github.com/pingcap/tidb/store/tikv" - "github.com/pingcap/tidb/store/tikv/oracle" - "github.com/pingcap/tidb/store/tikv/tikvrpc" -) - -type extractStartTsSuite struct { - store *tikv.KVStore -} - -var _ = SerialSuites(&extractStartTsSuite{}) - -func (s *extractStartTsSuite) SetUpTest(c *C) { - client, pdClient, cluster, err := unistore.New("") - c.Assert(err, IsNil) - unistore.BootstrapWithSingleStore(cluster) - store, err := tikv.NewTestTiKVStore(client, pdClient, nil, nil, 0) - c.Assert(err, IsNil) - probe := tikv.StoreProbe{KVStore: store} - probe.SetRegionCacheStore(2, tikvrpc.TiKV, 1, []*metapb.StoreLabel{ - { - Key: tikv.DCLabelKey, - Value: "local1", - }, - }) - probe.SetRegionCacheStore(3, tikvrpc.TiKV, 1, []*metapb.StoreLabel{ - { - Key: tikv.DCLabelKey, - Value: "Some Random Label", - }, - }) - probe.SetSafeTS(2, 102) - probe.SetSafeTS(3, 101) - s.store = probe.KVStore -} - -func (s *extractStartTsSuite) TestExtractStartTs(c *C) { - i := uint64(100) - // to prevent time change during test case execution - // we use failpoint to make it "fixed" - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp", "return(200)"), IsNil) - c.Assert(failpoint.Enable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp", `return(300)`), IsNil) - - cases := []struct { - expectedTS uint64 - option tikv.StartTSOption - }{ - // StartTS setted - {100, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: &i, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, - // PrevSec setted - {200, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: &i, MinStartTS: nil, MaxPrevSec: nil}}, - // MinStartTS setted, global - {101, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, - // MinStartTS setted, local - {102, tikv.StartTSOption{TxnScope: "local1", StartTS: nil, PrevSec: nil, MinStartTS: &i, MaxPrevSec: nil}}, - // MaxPrevSec setted - // however we need to add more cases to check the behavior when it fall backs to MinStartTS setted - // see `TestMaxPrevSecFallback` - {200, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, - // nothing setted - {300, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: nil}}, - } - for _, cs := range cases { - expected := cs.expectedTS - result, _ := tikv.ExtractStartTs(s.store, cs.option) - c.Assert(result, Equals, expected) - } - - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockStalenessTimestamp"), IsNil) - c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/MockCurrentTimestamp"), IsNil) -} - -func (s *extractStartTsSuite) TestMaxPrevSecFallback(c *C) { - probe := tikv.StoreProbe{KVStore: s.store} - probe.SetSafeTS(2, 0x8000000000000002) - probe.SetSafeTS(3, 0x8000000000000001) - i := uint64(100) - cases := []struct { - expectedTS uint64 - option tikv.StartTSOption - }{ - {0x8000000000000001, tikv.StartTSOption{TxnScope: oracle.GlobalTxnScope, StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, - {0x8000000000000002, tikv.StartTSOption{TxnScope: "local1", StartTS: nil, PrevSec: nil, MinStartTS: nil, MaxPrevSec: &i}}, - } - for _, cs := range cases { - result, _ := tikv.ExtractStartTs(s.store, cs.option) - c.Assert(result, Equals, cs.expectedTS) - } -} diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 988f6501be553..78fd2d5277a82 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -30,14 +30,12 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/failpoint" "github.com/pingcap/kvproto/pkg/kvrpcpb" - "github.com/pingcap/kvproto/pkg/metapb" tikverr "github.com/pingcap/tidb/store/tikv/error" tikv "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" "github.com/pingcap/tidb/store/tikv/oracle" "github.com/pingcap/tidb/store/tikv/retry" - "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/store/tikv/unionstore" "github.com/pingcap/tidb/store/tikv/util" "go.uber.org/zap" @@ -58,11 +56,8 @@ type SchemaAmender interface { // `TxnScope` must be set for each object // Every other fields are optional, but currently at most one of them can be set type StartTSOption struct { - TxnScope string - StartTS *uint64 - PrevSec *uint64 - MinStartTS *uint64 - MaxPrevSec *uint64 + TxnScope string + StartTS *uint64 } // DefaultStartTSOption creates a default StartTSOption, ie. Work in GlobalTxnScope and get start ts when got used @@ -70,30 +65,12 @@ func DefaultStartTSOption() StartTSOption { return StartTSOption{TxnScope: oracle.GlobalTxnScope} } -// SetMaxPrevSec returns a new StartTSOption with MaxPrevSec set to maxPrevSec -func (to StartTSOption) SetMaxPrevSec(maxPrevSec uint64) StartTSOption { - to.MaxPrevSec = &maxPrevSec - return to -} - -// SetMinStartTS returns a new StartTSOption with MinStartTS set to minStartTS -func (to StartTSOption) SetMinStartTS(minStartTS uint64) StartTSOption { - to.MinStartTS = &minStartTS - return to -} - -// SetStartTs returns a new StartTSOption with StartTS set to startTS -func (to StartTSOption) SetStartTs(startTS uint64) StartTSOption { +// SetStartTS returns a new StartTSOption with StartTS set to the given startTS +func (to StartTSOption) SetStartTS(startTS uint64) StartTSOption { to.StartTS = &startTS return to } -// SetPrevSec returns a new StartTSOption with PrevSec set to prevSec -func (to StartTSOption) SetPrevSec(prevSec uint64) StartTSOption { - to.PrevSec = &prevSec - return to -} - // SetTxnScope returns a new StartTSOption with TxnScope set to txnScope func (to StartTSOption) SetTxnScope(txnScope string) StartTSOption { to.TxnScope = txnScope @@ -135,68 +112,29 @@ type KVTxn struct { kvFilter KVFilter } -// ExtractStartTs use `option` to get the proper startTS for a transaction -func ExtractStartTs(store *KVStore, option StartTSOption) (uint64, error) { - var startTs uint64 - var err error +// ExtractStartTS use `option` to get the proper startTS for a transaction. +func ExtractStartTS(store *KVStore, option StartTSOption) (uint64, error) { if option.StartTS != nil { - startTs = *option.StartTS - } else if option.PrevSec != nil { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTs, err = store.getStalenessTimestamp(bo, option.TxnScope, *option.PrevSec) - } else if option.MinStartTS != nil { - stores := make([]*Store, 0) - allStores := store.regionCache.getStoresByType(tikvrpc.TiKV) - if option.TxnScope != oracle.GlobalTxnScope { - for _, store := range allStores { - if store.IsLabelsMatch([]*metapb.StoreLabel{ - { - Key: DCLabelKey, - Value: option.TxnScope, - }, - }) { - stores = append(stores, store) - } - } - } else { - stores = allStores - } - safeTS := store.getMinSafeTSByStores(stores) - startTs = *option.MinStartTS - // If the safeTS is larger than the minStartTS, we will use safeTS as StartTS, otherwise we will use - // minStartTS directly. - if startTs < safeTS { - startTs = safeTS - } - } else if option.MaxPrevSec != nil { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - minStartTS, err := store.getStalenessTimestamp(bo, option.TxnScope, *option.MaxPrevSec) - if err != nil { - return 0, errors.Trace(err) - } - option.MinStartTS = &minStartTS - return ExtractStartTs(store, option) - } else { - bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) - startTs, err = store.getTimestampWithRetry(bo, option.TxnScope) + return *option.StartTS, nil } - return startTs, err + bo := NewBackofferWithVars(context.Background(), tsoMaxBackoff, nil) + return store.getTimestampWithRetry(bo, option.TxnScope) } func newTiKVTxnWithOptions(store *KVStore, options StartTSOption) (*KVTxn, error) { if options.TxnScope == "" { options.TxnScope = oracle.GlobalTxnScope } - startTs, err := ExtractStartTs(store, options) + startTS, err := ExtractStartTS(store, options) if err != nil { return nil, errors.Trace(err) } - snapshot := newTiKVSnapshot(store, startTs, store.nextReplicaReadSeed()) + snapshot := newTiKVSnapshot(store, startTS, store.nextReplicaReadSeed()) newTiKVTxn := &KVTxn{ snapshot: snapshot, us: unionstore.NewUnionStore(snapshot), store: store, - startTS: startTs, + startTS: startTS, startTime: time.Now(), valid: true, vars: tikv.DefaultVars, diff --git a/util/mock/context.go b/util/mock/context.go index 4e9a9761f637a..9476d7295aa83 100644 --- a/util/mock/context.go +++ b/util/mock/context.go @@ -202,6 +202,11 @@ func (c *Context) NewTxn(context.Context) error { return nil } +// NewTxnWithStartTS implements the sessionctx.Context interface. +func (c *Context) NewTxnWithStartTS(ctx context.Context, startTS uint64) error { + return c.NewTxn(ctx) +} + // RefreshTxnCtx implements the sessionctx.Context interface. func (c *Context) RefreshTxnCtx(ctx context.Context) error { return errors.Trace(c.NewTxn(ctx)) @@ -218,7 +223,7 @@ func (c *Context) InitTxnWithStartTS(startTS uint64) error { return nil } if c.Store != nil { - txn, err := c.Store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(kv.GlobalTxnScope).SetStartTs(startTS)) + txn, err := c.Store.BeginWithOption(tikv.DefaultStartTSOption().SetTxnScope(kv.GlobalTxnScope).SetStartTS(startTS)) if err != nil { return errors.Trace(err) } @@ -227,11 +232,6 @@ func (c *Context) InitTxnWithStartTS(startTS uint64) error { return nil } -// NewTxnWithStalenessOption implements the sessionctx.Context interface. -func (c *Context) NewTxnWithStalenessOption(ctx context.Context, option sessionctx.StalenessTxnOption) error { - return c.NewTxn(ctx) -} - // GetStore gets the store of session. func (c *Context) GetStore() kv.Storage { return c.Store