diff --git a/distsql/request_builder_test.go b/distsql/request_builder_test.go index 18c1ee8fc24e4..ed921c391d6a3 100644 --- a/distsql/request_builder_test.go +++ b/distsql/request_builder_test.go @@ -25,7 +25,6 @@ import ( "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/chunk" @@ -324,7 +323,7 @@ func (s *testSuite) TestRequestBuilder1(c *C) { NotFillCache: false, SyncLog: false, Streaming: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -400,7 +399,7 @@ func (s *testSuite) TestRequestBuilder2(c *C) { NotFillCache: false, SyncLog: false, Streaming: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -447,7 +446,7 @@ func (s *testSuite) TestRequestBuilder3(c *C) { NotFillCache: false, SyncLog: false, Streaming: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -494,7 +493,7 @@ func (s *testSuite) TestRequestBuilder4(c *C) { Streaming: true, NotFillCache: false, SyncLog: false, - ReplicaRead: tikvstore.ReplicaReadLeader, + ReplicaRead: kv.ReplicaReadLeader, } c.Assert(actual, DeepEquals, expect) } @@ -577,10 +576,10 @@ func (s *testSuite) TestRequestBuilder6(c *C) { } func (s *testSuite) TestRequestBuilder7(c *C) { - for _, replicaRead := range []tikvstore.ReplicaReadType{ - tikvstore.ReplicaReadLeader, - tikvstore.ReplicaReadFollower, - tikvstore.ReplicaReadMixed, + for _, replicaRead := range []kv.ReplicaReadType{ + kv.ReplicaReadLeader, + kv.ReplicaReadFollower, + kv.ReplicaReadMixed, } { vars := variable.NewSessionVars() vars.SetReplicaRead(replicaRead) @@ -626,7 +625,6 @@ func (s *testSuite) TestRequestBuilder8(c *C) { IsolationLevel: 0, Priority: 0, MemTracker: (*memory.Tracker)(nil), - ReplicaRead: 0x1, SchemaVar: 10000, } c.Assert(actual, DeepEquals, expect) diff --git a/executor/analyze.go b/executor/analyze.go index fec55d870bf95..633f6e4dcc69e 100644 --- a/executor/analyze.go +++ b/executor/analyze.go @@ -45,7 +45,6 @@ import ( "github.com/pingcap/tidb/sessionctx/variable" "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/table" "github.com/pingcap/tidb/tablecodec" "github.com/pingcap/tidb/types" @@ -1322,7 +1321,7 @@ func (e *AnalyzeFastExec) handleScanIter(iter kv.Iterator) (scanKeysSize int, er func (e *AnalyzeFastExec) handleScanTasks(bo *tikv.Backoffer) (keysSize int, err error) { snapshot := e.ctx.GetStore().GetSnapshot(kv.MaxVersion) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } for _, t := range e.scanTasks { iter, err := snapshot.Iter(kv.Key(t.StartKey), kv.Key(t.EndKey)) @@ -1345,7 +1344,7 @@ func (e *AnalyzeFastExec) handleSampTasks(workID int, step uint32, err *error) { snapshot.SetOption(kv.IsolationLevel, kv.RC) snapshot.SetOption(kv.Priority, kv.PriorityLow) if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } rander := rand.New(rand.NewSource(e.randSeed)) diff --git a/executor/analyze_test.go b/executor/analyze_test.go index 39ec524dbacc4..b527875c1498b 100644 --- a/executor/analyze_test.go +++ b/executor/analyze_test.go @@ -29,6 +29,7 @@ import ( "github.com/pingcap/tidb/domain" "github.com/pingcap/tidb/executor" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/planner/core" "github.com/pingcap/tidb/session" "github.com/pingcap/tidb/sessionctx" @@ -38,7 +39,6 @@ import ( "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/tikvrpc" "github.com/pingcap/tidb/table" @@ -121,7 +121,7 @@ func (s *testSuite1) TestAnalyzeReplicaReadFollower(c *C) { tk.MustExec("drop table if exists t") tk.MustExec("create table t(a int)") ctx := tk.Se.(sessionctx.Context) - ctx.GetSessionVars().SetReplicaRead(tikvstore.ReplicaReadFollower) + ctx.GetSessionVars().SetReplicaRead(kv.ReplicaReadFollower) tk.MustExec("analyze table t") } diff --git a/executor/batch_point_get.go b/executor/batch_point_get.go index 23debe37404ee..2137884c69745 100644 --- a/executor/batch_point_get.go +++ b/executor/batch_point_get.go @@ -117,7 +117,7 @@ func (e *BatchPointGetExec) Open(context.Context) error { e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness diff --git a/executor/point_get.go b/executor/point_get.go index 8857a4d253fd0..fc8326555bf01 100644 --- a/executor/point_get.go +++ b/executor/point_get.go @@ -148,7 +148,7 @@ func (e *PointGetExecutor) Open(context.Context) error { e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats) } if e.ctx.GetSessionVars().GetReplicaRead().IsFollowerRead() { - e.snapshot.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + e.snapshot.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } e.snapshot.SetOption(kv.TaskID, e.ctx.GetSessionVars().StmtCtx.TaskID) isStaleness := e.ctx.GetSessionVars().TxnCtx.IsStaleness diff --git a/kv/kv.go b/kv/kv.go index e5ab4eed6f812..572fe104024bc 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -273,7 +273,7 @@ type Request struct { // call would not corresponds to a whole region result. Streaming bool // ReplicaRead is used for reading data from replicas, only follower is supported at this time. - ReplicaRead tikvstore.ReplicaReadType + ReplicaRead ReplicaReadType // StoreType represents this request is sent to the which type of store. StoreType StoreType // Cacheable is true if the request can be cached. Currently only deterministic DAG requests can be cached. diff --git a/kv/option.go b/kv/option.go index 5b04dfba06c95..dc0d700666d5a 100644 --- a/kv/option.go +++ b/kv/option.go @@ -60,3 +60,20 @@ const ( // MatchStoreLabels indicates the labels the store should be matched MatchStoreLabels ) + +// ReplicaReadType is the type of replica to read data from +type ReplicaReadType byte + +const ( + // ReplicaReadLeader stands for 'read from leader'. + ReplicaReadLeader ReplicaReadType = iota + // ReplicaReadFollower stands for 'read from follower'. + ReplicaReadFollower + // ReplicaReadMixed stands for 'read from leader and follower and learner'. + ReplicaReadMixed +) + +// IsFollowerRead checks if follower is going to be used to read data. +func (r ReplicaReadType) IsFollowerRead() bool { + return r != ReplicaReadLeader +} diff --git a/planner/optimize.go b/planner/optimize.go index 4e0f7334ae5ae..ec9bfef67d0a7 100644 --- a/planner/optimize.go +++ b/planner/optimize.go @@ -36,7 +36,6 @@ import ( "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/sessionctx/variable" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/types" "github.com/pingcap/tidb/util/hint" "github.com/pingcap/tidb/util/logutil" @@ -533,7 +532,7 @@ func handleStmtHints(hints []*ast.TableOptimizerHint) (stmtHints stmtctx.StmtHin warns = append(warns, warn) } stmtHints.HasReplicaReadHint = true - stmtHints.ReplicaRead = byte(tikvstore.ReplicaReadFollower) + stmtHints.ReplicaRead = byte(kv.ReplicaReadFollower) } // Handle MAX_EXECUTION_TIME if maxExecutionTimeCnt != 0 { diff --git a/session/session.go b/session/session.go index 8fc7c4c37eac8..af3f41c863dc0 100644 --- a/session/session.go +++ b/session/session.go @@ -69,7 +69,6 @@ import ( "github.com/pingcap/tidb/statistics" "github.com/pingcap/tidb/statistics/handle" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" "github.com/pingcap/tidb/tablecodec" @@ -1891,7 +1890,7 @@ func (s *session) Txn(active bool) (kv.Transaction, error) { s.sessionVars.TxnCtx.CouldRetry = s.isTxnRetryable() s.txn.SetVars(s.sessionVars.KVVars) if s.sessionVars.GetReplicaRead().IsFollowerRead() { - s.txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + s.txn.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } } return &s.txn, nil @@ -1955,7 +1954,7 @@ func (s *session) NewTxn(ctx context.Context) error { } txn.SetVars(s.sessionVars.KVVars) if s.GetSessionVars().GetReplicaRead().IsFollowerRead() { - txn.SetOption(kv.ReplicaRead, tikvstore.ReplicaReadFollower) + txn.SetOption(kv.ReplicaRead, kv.ReplicaReadFollower) } s.txn.changeInvalidToValid(txn) is := domain.GetDomain(s).InfoSchema() diff --git a/session/session_test.go b/session/session_test.go index a8861dfddd79c..4870215f33c9e 100644 --- a/session/session_test.go +++ b/session/session_test.go @@ -51,7 +51,6 @@ import ( "github.com/pingcap/tidb/store/mockstore" "github.com/pingcap/tidb/store/mockstore/mockcopr" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" "github.com/pingcap/tidb/store/tikv/mockstore/cluster" "github.com/pingcap/tidb/store/tikv/oracle" tikvutil "github.com/pingcap/tidb/store/tikv/util" @@ -3064,11 +3063,11 @@ func (s *testSessionSuite2) TestReplicaRead(c *C) { tk := testkit.NewTestKit(c, s.store) tk.Se, err = session.CreateSession4Test(s.store) c.Assert(err, IsNil) - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadLeader) tk.MustExec("set @@tidb_replica_read = 'follower';") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) tk.MustExec("set @@tidb_replica_read = 'leader';") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadLeader) } func (s *testSessionSuite3) TestIsolationRead(c *C) { @@ -3153,12 +3152,12 @@ func (s *testSessionSuite2) TestStmtHints(c *C) { c.Assert(tk.Se.GetSessionVars().GetEnableCascadesPlanner(), IsTrue) // Test READ_CONSISTENT_REPLICA hint - tk.Se.GetSessionVars().SetReplicaRead(tikvstore.ReplicaReadLeader) + tk.Se.GetSessionVars().SetReplicaRead(kv.ReplicaReadLeader) tk.MustExec("select /*+ READ_CONSISTENT_REPLICA() */ 1;") - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) tk.MustExec("select /*+ READ_CONSISTENT_REPLICA(), READ_CONSISTENT_REPLICA() */ 1;") c.Assert(tk.Se.GetSessionVars().StmtCtx.GetWarnings(), HasLen, 1) - c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(tk.Se.GetSessionVars().GetReplicaRead(), Equals, kv.ReplicaReadFollower) } func (s *testSessionSuite3) TestPessimisticLockOnPartition(c *C) { diff --git a/sessionctx/variable/session.go b/sessionctx/variable/session.go index 97597997b36f5..0c6c74d90a26d 100644 --- a/sessionctx/variable/session.go +++ b/sessionctx/variable/session.go @@ -723,7 +723,7 @@ type SessionVars struct { enableIndexMerge bool // replicaRead is used for reading data from replicas, only follower is supported at this time. - replicaRead tikvstore.ReplicaReadType + replicaRead kv.ReplicaReadType // IsolationReadEngines is used to isolation read, tidb only read from the stores whose engine type is in the engines. IsolationReadEngines map[kv.StoreType]struct{} @@ -1029,7 +1029,7 @@ func NewSessionVars() *SessionVars { WaitSplitRegionTimeout: DefWaitSplitRegionTimeout, enableIndexMerge: false, EnableNoopFuncs: DefTiDBEnableNoopFuncs, - replicaRead: tikvstore.ReplicaReadLeader, + replicaRead: kv.ReplicaReadLeader, AllowRemoveAutoInc: DefTiDBAllowRemoveAutoInc, UsePlanBaselines: DefTiDBUsePlanBaselines, EvolvePlanBaselines: DefTiDBEvolvePlanBaselines, @@ -1179,15 +1179,15 @@ func (s *SessionVars) SetEnableIndexMerge(val bool) { } // GetReplicaRead get ReplicaRead from sql hints and SessionVars.replicaRead. -func (s *SessionVars) GetReplicaRead() tikvstore.ReplicaReadType { +func (s *SessionVars) GetReplicaRead() kv.ReplicaReadType { if s.StmtCtx.HasReplicaReadHint { - return tikvstore.ReplicaReadType(s.StmtCtx.ReplicaRead) + return kv.ReplicaReadType(s.StmtCtx.ReplicaRead) } return s.replicaRead } // SetReplicaRead set SessionVars.replicaRead. -func (s *SessionVars) SetReplicaRead(val tikvstore.ReplicaReadType) { +func (s *SessionVars) SetReplicaRead(val kv.ReplicaReadType) { s.replicaRead = val } diff --git a/sessionctx/variable/sysvar.go b/sessionctx/variable/sysvar.go index 574e649656205..98518fe4af0f0 100644 --- a/sessionctx/variable/sysvar.go +++ b/sessionctx/variable/sysvar.go @@ -1246,11 +1246,11 @@ var defaultSysVars = []*SysVar{ }}, {Scope: ScopeSession, Name: TiDBReplicaRead, Value: "leader", Type: TypeEnum, PossibleValues: []string{"leader", "follower", "leader-and-follower"}, SetSession: func(s *SessionVars, val string) error { if strings.EqualFold(val, "follower") { - s.SetReplicaRead(tikvstore.ReplicaReadFollower) + s.SetReplicaRead(kv.ReplicaReadFollower) } else if strings.EqualFold(val, "leader-and-follower") { - s.SetReplicaRead(tikvstore.ReplicaReadMixed) + s.SetReplicaRead(kv.ReplicaReadMixed) } else if strings.EqualFold(val, "leader") || len(val) == 0 { - s.SetReplicaRead(tikvstore.ReplicaReadLeader) + s.SetReplicaRead(kv.ReplicaReadLeader) } return nil }}, diff --git a/sessionctx/variable/varsutil_test.go b/sessionctx/variable/varsutil_test.go index 90507f36539d6..1d8e629b7df4a 100644 --- a/sessionctx/variable/varsutil_test.go +++ b/sessionctx/variable/varsutil_test.go @@ -23,7 +23,7 @@ import ( "github.com/pingcap/parser/mysql" "github.com/pingcap/parser/terror" "github.com/pingcap/tidb/config" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" + "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/util/testleak" ) @@ -431,19 +431,19 @@ func (s *testVarsutilSuite) TestVarsutil(c *C) { val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "follower") - c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadFollower) + c.Assert(v.GetReplicaRead(), Equals, kv.ReplicaReadFollower) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader") - c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadLeader) + c.Assert(v.GetReplicaRead(), Equals, kv.ReplicaReadLeader) err = SetSessionSystemVar(v, TiDBReplicaRead, "leader-and-follower") c.Assert(err, IsNil) val, err = GetSessionOrGlobalSystemVar(v, TiDBReplicaRead) c.Assert(err, IsNil) c.Assert(val, Equals, "leader-and-follower") - c.Assert(v.GetReplicaRead(), Equals, tikvstore.ReplicaReadMixed) + c.Assert(v.GetReplicaRead(), Equals, kv.ReplicaReadMixed) err = SetSessionSystemVar(v, TiDBEnableStmtSummary, "ON") c.Assert(err, IsNil) diff --git a/store/copr/coprocessor.go b/store/copr/coprocessor.go index e9d9e6b8f1ebb..989a6d835ce0f 100644 --- a/store/copr/coprocessor.go +++ b/store/copr/coprocessor.go @@ -37,6 +37,7 @@ import ( tidbmetrics "github.com/pingcap/tidb/metrics" "github.com/pingcap/tidb/store/driver/backoff" derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/store/tikv" "github.com/pingcap/tidb/store/tikv/logutil" "github.com/pingcap/tidb/store/tikv/metrics" @@ -697,7 +698,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch } } - req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, worker.req.ReplicaRead, &worker.replicaReadSeed, kvrpcpb.Context{ + req := tikvrpc.NewReplicaReadRequest(task.cmdType, &copReq, options.GetTiKVReplicaReadType(worker.req.ReplicaRead), &worker.replicaReadSeed, kvrpcpb.Context{ IsolationLevel: isolationLevelToPB(worker.req.IsolationLevel), Priority: priorityToPB(worker.req.Priority), NotFillCache: worker.req.NotFillCache, diff --git a/store/driver/options/options.go b/store/driver/options/options.go new file mode 100644 index 0000000000000..dc16f7793ed91 --- /dev/null +++ b/store/driver/options/options.go @@ -0,0 +1,32 @@ +// 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 options + +import ( + "github.com/pingcap/tidb/kv" + storekv "github.com/pingcap/tidb/store/tikv/kv" +) + +// GetTiKVReplicaReadType maps kv.ReplicaReadType to tikv/kv.ReplicaReadType. +func GetTiKVReplicaReadType(t kv.ReplicaReadType) storekv.ReplicaReadType { + switch t { + case kv.ReplicaReadLeader: + return storekv.ReplicaReadLeader + case kv.ReplicaReadFollower: + return storekv.ReplicaReadFollower + case kv.ReplicaReadMixed: + return storekv.ReplicaReadMixed + } + return 0 +} diff --git a/store/driver/txn/snapshot.go b/store/driver/txn/snapshot.go index 405067f5e082b..6692f45a749a3 100644 --- a/store/driver/txn/snapshot.go +++ b/store/driver/txn/snapshot.go @@ -20,8 +20,8 @@ import ( "github.com/pingcap/kvproto/pkg/metapb" "github.com/pingcap/tidb/kv" derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/store/tikv" - tikvstore "github.com/pingcap/tidb/store/tikv/kv" ) type tikvSnapshot struct { @@ -76,7 +76,8 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) { case kv.SnapshotTS: s.KVSnapshot.SetSnapshotTS(val.(uint64)) case kv.ReplicaRead: - s.KVSnapshot.SetReplicaRead(val.(tikvstore.ReplicaReadType)) + t := options.GetTiKVReplicaReadType(val.(kv.ReplicaReadType)) + s.KVSnapshot.SetReplicaRead(t) case kv.SampleStep: s.KVSnapshot.SetSampleStep(val.(uint32)) case kv.TaskID: diff --git a/store/driver/txn/txn_driver.go b/store/driver/txn/txn_driver.go index 7c940561f82fe..7f05f80139c12 100644 --- a/store/driver/txn/txn_driver.go +++ b/store/driver/txn/txn_driver.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/kv" "github.com/pingcap/tidb/sessionctx/binloginfo" derr "github.com/pingcap/tidb/store/driver/error" + "github.com/pingcap/tidb/store/driver/options" "github.com/pingcap/tidb/store/tikv" tikverr "github.com/pingcap/tidb/store/tikv/error" tikvstore "github.com/pingcap/tidb/store/tikv/kv" @@ -143,7 +144,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) { case kv.SnapshotTS: txn.KVTxn.GetSnapshot().SetSnapshotTS(val.(uint64)) case kv.ReplicaRead: - txn.KVTxn.GetSnapshot().SetReplicaRead(val.(tikvstore.ReplicaReadType)) + t := options.GetTiKVReplicaReadType(val.(kv.ReplicaReadType)) + txn.KVTxn.GetSnapshot().SetReplicaRead(t) case kv.TaskID: txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64)) case kv.InfoSchema: diff --git a/store/tikv/kv/store_vars.go b/store/tikv/kv/store_vars.go index 5f65f927bffb9..02d87018213a9 100644 --- a/store/tikv/kv/store_vars.go +++ b/store/tikv/kv/store_vars.go @@ -25,15 +25,14 @@ type ReplicaReadType byte const ( // ReplicaReadLeader stands for 'read from leader'. - ReplicaReadLeader ReplicaReadType = 1 << iota + ReplicaReadLeader ReplicaReadType = iota // ReplicaReadFollower stands for 'read from follower'. ReplicaReadFollower // ReplicaReadMixed stands for 'read from leader and follower and learner'. ReplicaReadMixed ) -// IsFollowerRead checks if leader is going to be used to read data. +// IsFollowerRead checks if follower is going to be used to read data. func (r ReplicaReadType) IsFollowerRead() bool { - // In some cases the default value is 0, which should be treated as `ReplicaReadLeader`. - return r != ReplicaReadLeader && r != 0 + return r != ReplicaReadLeader }