Skip to content

Commit

Permalink
store/tikv: remove use of CollectRuntimeStats option in store/tikv (#…
Browse files Browse the repository at this point in the history
  • Loading branch information
disksing authored May 13, 2021
1 parent cc83cc5 commit 9692c13
Show file tree
Hide file tree
Showing 5 changed files with 29 additions and 26 deletions.
11 changes: 9 additions & 2 deletions store/driver/txn/snapshot.go
Original file line number Diff line number Diff line change
Expand Up @@ -81,12 +81,19 @@ func (s *tikvSnapshot) SetOption(opt int, val interface{}) {
s.KVSnapshot.SetSampleStep(val.(uint32))
case tikvstore.TaskID:
s.KVSnapshot.SetTaskID(val.(uint64))
case tikvstore.CollectRuntimeStats:
s.KVSnapshot.SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats))
case tikvstore.IsStalenessReadOnly:
s.KVSnapshot.SetIsStatenessReadOnly(val.(bool))
case tikvstore.MatchStoreLabels:
s.KVSnapshot.SetMatchStoreLabels(val.([]*metapb.StoreLabel))
default:
s.KVSnapshot.SetOption(opt, val)
}
}

func (s *tikvSnapshot) DelOption(opt int) {
switch opt {
case tikvstore.CollectRuntimeStats:
s.KVSnapshot.SetRuntimeStats(nil)
}
}

Expand Down
11 changes: 11 additions & 0 deletions store/driver/txn/txn_driver.go
Original file line number Diff line number Diff line change
Expand Up @@ -152,6 +152,8 @@ func (txn *tikvTxn) SetOption(opt int, val interface{}) {
txn.KVTxn.GetSnapshot().SetTaskID(val.(uint64))
case tikvstore.InfoSchema:
txn.SetSchemaVer(val.(tikv.SchemaVer))
case tikvstore.CollectRuntimeStats:
txn.KVTxn.GetSnapshot().SetRuntimeStats(val.(*tikv.SnapshotRuntimeStats))
case tikvstore.SchemaAmender:
txn.SetSchemaAmender(val.(tikv.SchemaAmender))
case tikvstore.SampleStep:
Expand Down Expand Up @@ -186,6 +188,15 @@ func (txn *tikvTxn) GetOption(opt int) interface{} {
}
}

func (txn *tikvTxn) DelOption(opt int) {
switch opt {
case tikvstore.CollectRuntimeStats:
txn.KVTxn.GetSnapshot().SetRuntimeStats(nil)
default:
txn.KVTxn.DelOption(opt)
}
}

// SetVars sets variables to the transaction.
func (txn *tikvTxn) SetVars(vars interface{}) {
if vs, ok := vars.(*tikv.Variables); ok {
Expand Down
29 changes: 8 additions & 21 deletions store/tikv/snapshot.go
Original file line number Diff line number Diff line change
Expand Up @@ -561,27 +561,6 @@ func (s *KVSnapshot) IterReverse(k []byte) (unionstore.Iterator, error) {
return scanner, errors.Trace(err)
}

// SetOption sets an option with a value, when val is nil, uses the default
// value of this option. Only ReplicaRead is supported for snapshot
func (s *KVSnapshot) SetOption(opt int, val interface{}) {
switch opt {
case kv.CollectRuntimeStats:
s.mu.Lock()
s.mu.stats = val.(*SnapshotRuntimeStats)
s.mu.Unlock()
}
}

// DelOption deletes an option.
func (s *KVSnapshot) DelOption(opt int) {
switch opt {
case kv.CollectRuntimeStats:
s.mu.Lock()
s.mu.stats = nil
s.mu.Unlock()
}
}

// SetNotFillCache indicates whether tikv should skip filling cache when
// loading data.
func (s *KVSnapshot) SetNotFillCache(b bool) {
Expand Down Expand Up @@ -623,6 +602,14 @@ func (s *KVSnapshot) SetTaskID(id uint64) {
s.mu.taskID = id
}

// SetRuntimeStats sets the stats to collect runtime statistics.
// Set it to nil to clear stored stats.
func (s *KVSnapshot) SetRuntimeStats(stats *SnapshotRuntimeStats) {
s.mu.Lock()
defer s.mu.Unlock()
s.mu.stats = stats
}

// SetIsStatenessReadOnly indicates whether the transaction is staleness read only transaction
func (s *KVSnapshot) SetIsStatenessReadOnly(b bool) {
s.mu.Lock()
Expand Down
3 changes: 1 addition & 2 deletions store/tikv/tests/snapshot_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,6 @@ import (
pb "github.com/pingcap/kvproto/pkg/kvrpcpb"
"github.com/pingcap/tidb/store/tikv"
tikverr "github.com/pingcap/tidb/store/tikv/error"
"github.com/pingcap/tidb/store/tikv/kv"
"github.com/pingcap/tidb/store/tikv/logutil"
"github.com/pingcap/tidb/store/tikv/tikvrpc"
"go.uber.org/zap"
Expand Down Expand Up @@ -270,7 +269,7 @@ func (s *testSnapshotSuite) TestSnapshotRuntimeStats(c *C) {
tikv.RecordRegionRequestRuntimeStats(reqStats.Stats, tikvrpc.CmdGet, time.Second)
tikv.RecordRegionRequestRuntimeStats(reqStats.Stats, tikvrpc.CmdGet, time.Millisecond)
snapshot := s.store.GetSnapshot(0)
snapshot.SetOption(kv.CollectRuntimeStats, &tikv.SnapshotRuntimeStats{})
snapshot.SetRuntimeStats(&tikv.SnapshotRuntimeStats{})
snapshot.MergeRegionRequestStats(reqStats.Stats)
snapshot.MergeRegionRequestStats(reqStats.Stats)
bo := tikv.NewBackofferWithVars(context.Background(), 2000, nil)
Expand Down
1 change: 0 additions & 1 deletion store/tikv/txn.go
Original file line number Diff line number Diff line change
Expand Up @@ -224,7 +224,6 @@ func (txn *KVTxn) Delete(k []byte) error {
// value of this option.
func (txn *KVTxn) SetOption(opt int, val interface{}) {
txn.us.SetOption(opt, val)
txn.snapshot.SetOption(opt, val)
}

// GetOption returns the option
Expand Down

0 comments on commit 9692c13

Please sign in to comment.