Skip to content

Commit

Permalink
*/backoff: make backoff type as string instead of interface fmt.Strin…
Browse files Browse the repository at this point in the history
…ger (#24810)
  • Loading branch information
AndreMouche committed May 25, 2021
1 parent 98f0d76 commit 8c25a00
Show file tree
Hide file tree
Showing 6 changed files with 41 additions and 62 deletions.
12 changes: 8 additions & 4 deletions store/tikv/retry/backoff.go
Original file line number Diff line number Diff line change
Expand Up @@ -41,7 +41,7 @@ type Backoffer struct {
maxSleep int
totalSleep int
errors []error
configs []fmt.Stringer
configs []*Config
vars *kv.Variables
noop bool

Expand Down Expand Up @@ -172,7 +172,7 @@ func (b *Backoffer) BackoffWithCfgAndMaxSleep(cfg *Config, maxSleepMs int, err e
}
logutil.BgLogger().Warn(errMsg)
// Use the first backoff type to generate a MySQL error.
return b.configs[0].(*Config).err
return b.configs[0].err
}

// Lazy initialize.
Expand Down Expand Up @@ -265,8 +265,12 @@ func (b *Backoffer) GetTotalSleep() int {
}

// GetTypes returns type list.
func (b *Backoffer) GetTypes() []fmt.Stringer {
return b.configs
func (b *Backoffer) GetTypes() []string {
typs := make([]string, 0, len(b.configs))
for _, cfg := range b.configs {
typs = append(typs, cfg.String())
}
return typs
}

// GetCtx returns the binded context.
Expand Down
9 changes: 4 additions & 5 deletions store/tikv/util/execdetails.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ package util

import (
"bytes"
"fmt"
"math"
"strconv"
"sync"
Expand Down Expand Up @@ -50,7 +49,7 @@ type CommitDetails struct {
CommitBackoffTime int64
Mu struct {
sync.Mutex
BackoffTypes []fmt.Stringer
BackoffTypes []string
}
ResolveLockTime int64
WriteKeys int
Expand Down Expand Up @@ -90,7 +89,7 @@ func (cd *CommitDetails) Clone() *CommitDetails {
PrewriteRegionNum: cd.PrewriteRegionNum,
TxnRetry: cd.TxnRetry,
}
commit.Mu.BackoffTypes = append([]fmt.Stringer{}, cd.Mu.BackoffTypes...)
commit.Mu.BackoffTypes = append([]string{}, cd.Mu.BackoffTypes...)
return commit
}

Expand All @@ -103,7 +102,7 @@ type LockKeysDetails struct {
BackoffTime int64
Mu struct {
sync.Mutex
BackoffTypes []fmt.Stringer
BackoffTypes []string
}
LockRPCTime int64
LockRPCCount int64
Expand Down Expand Up @@ -135,7 +134,7 @@ func (ld *LockKeysDetails) Clone() *LockKeysDetails {
LockRPCCount: ld.LockRPCCount,
RetryCount: ld.RetryCount,
}
lock.Mu.BackoffTypes = append([]fmt.Stringer{}, ld.Mu.BackoffTypes...)
lock.Mu.BackoffTypes = append([]string{}, ld.Mu.BackoffTypes...)
return lock
}

Expand Down
5 changes: 2 additions & 3 deletions util/execdetails/execdetails.go
Original file line number Diff line number Diff line change
Expand Up @@ -892,14 +892,13 @@ func (e *RuntimeStatsWithCommit) String() string {
return buf.String()
}

func (e *RuntimeStatsWithCommit) formatBackoff(backoffTypes []fmt.Stringer) string {
func (e *RuntimeStatsWithCommit) formatBackoff(backoffTypes []string) string {
if len(backoffTypes) == 0 {
return ""
}
tpMap := make(map[string]struct{})
tpArray := []string{}
for _, tp := range backoffTypes {
tpStr := tp.String()
for _, tpStr := range backoffTypes {
_, ok := tpMap[tpStr]
if ok {
continue
Expand Down
44 changes: 11 additions & 33 deletions util/execdetails/execdetails_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,15 +14,13 @@
package execdetails

import (
"fmt"
"strconv"
"sync"
"testing"
"time"

. "github.com/pingcap/check"
"github.com/pingcap/tidb/store/tikv/util"
"github.com/pingcap/tidb/util/stringutil"
"github.com/pingcap/tipb/go-tipb"
)

Expand All @@ -43,14 +41,10 @@ func TestString(t *testing.T) {
CommitBackoffTime: int64(time.Second),
Mu: struct {
sync.Mutex
BackoffTypes []fmt.Stringer
}{BackoffTypes: []fmt.Stringer{
stringutil.MemoizeStr(func() string {
return "backoff1"
}),
stringutil.MemoizeStr(func() string {
return "backoff2"
}),
BackoffTypes []string
}{BackoffTypes: []string{
"backoff1",
"backoff2",
}},
ResolveLockTime: 1000000000, // 10^9 ns = 1s
WriteKeys: 1,
Expand Down Expand Up @@ -212,18 +206,8 @@ func TestRuntimeStatsWithCommit(t *testing.T) {
CommitBackoffTime: int64(time.Second),
Mu: struct {
sync.Mutex
BackoffTypes []fmt.Stringer
}{BackoffTypes: []fmt.Stringer{
stringutil.MemoizeStr(func() string {
return "backoff1"
}),
stringutil.MemoizeStr(func() string {
return "backoff2"
}),
stringutil.MemoizeStr(func() string {
return "backoff1"
}),
}},
BackoffTypes []string
}{BackoffTypes: []string{"backoff1", "backoff2", "backoff1"}},
ResolveLockTime: int64(time.Second),
WriteKeys: 3,
WriteSize: 66,
Expand All @@ -245,17 +229,11 @@ func TestRuntimeStatsWithCommit(t *testing.T) {
BackoffTime: int64(time.Second * 3),
Mu: struct {
sync.Mutex
BackoffTypes []fmt.Stringer
}{BackoffTypes: []fmt.Stringer{
stringutil.MemoizeStr(func() string {
return "backoff4"
}),
stringutil.MemoizeStr(func() string {
return "backoff5"
}),
stringutil.MemoizeStr(func() string {
return "backoff5"
}),
BackoffTypes []string
}{BackoffTypes: []string{
"backoff4",
"backoff5",
"backoff5",
}},
LockRPCTime: int64(time.Second * 5),
LockRPCCount: 50,
Expand Down
8 changes: 4 additions & 4 deletions util/stmtsummary/statement_summary.go
Original file line number Diff line number Diff line change
Expand Up @@ -172,7 +172,7 @@ type stmtSummaryByDigestElement struct {
sumTxnRetry int64
maxTxnRetry int
sumBackoffTimes int64
backoffTypes map[fmt.Stringer]int
backoffTypes map[string]int
authUsers map[string]struct{}
// other
sumMem int64
Expand Down Expand Up @@ -635,7 +635,7 @@ func newStmtSummaryByDigestElement(sei *StmtExecInfo, beginTime int64, intervalS
minLatency: sei.TotalLatency,
firstSeen: sei.StartTime,
lastSeen: sei.StartTime,
backoffTypes: make(map[fmt.Stringer]int),
backoffTypes: make(map[string]int),
authUsers: make(map[string]struct{}),
planInCache: false,
planCacheHits: 0,
Expand Down Expand Up @@ -971,9 +971,9 @@ func formatSQL(sql string) string {
}

// Format the backoffType map to a string or nil.
func formatBackoffTypes(backoffMap map[fmt.Stringer]int) interface{} {
func formatBackoffTypes(backoffMap map[string]int) interface{} {
type backoffStat struct {
backoffType fmt.Stringer
backoffType string
count int
}

Expand Down
25 changes: 12 additions & 13 deletions util/stmtsummary/statement_summary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,7 +30,6 @@ import (
"github.com/pingcap/tidb/store/tikv/util"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/stringutil"
)

var _ = Suite(&testStmtSummarySuite{})
Expand Down Expand Up @@ -63,7 +62,7 @@ func TestT(t *testing.T) {
}

const (
boTxnLockName = stringutil.StringerStr("txnlock")
boTxnLockName = "txnlock"
)

// Test stmtSummaryByDigest.AddStatement.
Expand All @@ -77,7 +76,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) {

// first statement
stmtExecInfo1 := generateAnyExecInfo()
stmtExecInfo1.ExecDetail.CommitDetail.Mu.BackoffTypes = make([]fmt.Stringer, 0)
stmtExecInfo1.ExecDetail.CommitDetail.Mu.BackoffTypes = make([]string, 0)
key := &stmtSummaryByDigestKey{
schemaName: stmtExecInfo1.SchemaName,
digest: stmtExecInfo1.Digest,
Expand Down Expand Up @@ -133,7 +132,7 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) {
maxPrewriteRegionNum: stmtExecInfo1.ExecDetail.CommitDetail.PrewriteRegionNum,
sumTxnRetry: int64(stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry),
maxTxnRetry: stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry,
backoffTypes: make(map[fmt.Stringer]int),
backoffTypes: make(map[string]int),
sumMem: stmtExecInfo1.MemMax,
maxMem: stmtExecInfo1.MemMax,
sumDisk: stmtExecInfo1.DiskMax,
Expand Down Expand Up @@ -194,9 +193,9 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) {
CommitBackoffTime: 1000,
Mu: struct {
sync.Mutex
BackoffTypes []fmt.Stringer
BackoffTypes []string
}{
BackoffTypes: []fmt.Stringer{boTxnLockName},
BackoffTypes: []string{boTxnLockName},
},
ResolveLockTime: 10000,
WriteKeys: 100000,
Expand Down Expand Up @@ -321,9 +320,9 @@ func (s *testStmtSummarySuite) TestAddStatement(c *C) {
CommitBackoffTime: 100,
Mu: struct {
sync.Mutex
BackoffTypes []fmt.Stringer
BackoffTypes []string
}{
BackoffTypes: []fmt.Stringer{boTxnLockName},
BackoffTypes: []string{boTxnLockName},
},
ResolveLockTime: 1000,
WriteKeys: 10000,
Expand Down Expand Up @@ -577,9 +576,9 @@ func generateAnyExecInfo() *StmtExecInfo {
CommitBackoffTime: 200,
Mu: struct {
sync.Mutex
BackoffTypes []fmt.Stringer
BackoffTypes []string
}{
BackoffTypes: []fmt.Stringer{boTxnLockName},
BackoffTypes: []string{boTxnLockName},
},
ResolveLockTime: 2000,
WriteKeys: 20000,
Expand Down Expand Up @@ -961,12 +960,12 @@ func (s *testStmtSummarySuite) TestGetMoreThanOnceBindableStmt(c *C) {

// Test `formatBackoffTypes`.
func (s *testStmtSummarySuite) TestFormatBackoffTypes(c *C) {
backoffMap := make(map[fmt.Stringer]int)
backoffMap := make(map[string]int)
c.Assert(formatBackoffTypes(backoffMap), IsNil)
bo1 := stringutil.StringerStr("pdrpc")
bo1 := "pdrpc"
backoffMap[bo1] = 1
c.Assert(formatBackoffTypes(backoffMap), Equals, "pdrpc:1")
bo2 := stringutil.StringerStr("txnlock")
bo2 := "txnlock"
backoffMap[bo2] = 2

c.Assert(formatBackoffTypes(backoffMap), Equals, "txnlock:2,pdrpc:1")
Expand Down

0 comments on commit 8c25a00

Please sign in to comment.