From 08c63bf3370921c11adb849cfd5597ed11889086 Mon Sep 17 00:00:00 2001 From: ti-srebot <66930949+ti-srebot@users.noreply.github.com> Date: Sat, 10 Oct 2020 16:42:59 +0800 Subject: [PATCH] executor: add pessimistic lock keys runtime information (#19547) (#20199) --- executor/adapter.go | 28 ++-- executor/executor.go | 8 +- executor/executor_test.go | 37 +++++ kv/kv.go | 2 + sessionctx/stmtctx/stmtctx.go | 11 ++ store/tikv/2pc.go | 17 ++- store/tikv/txn.go | 18 +++ util/execdetails/execdetails.go | 218 +++++++++++++++++++-------- util/execdetails/execdetails_test.go | 34 ++++- 9 files changed, 293 insertions(+), 80 deletions(-) diff --git a/executor/adapter.go b/executor/adapter.go index a55f7a760de98..329198cc58aef 100644 --- a/executor/adapter.go +++ b/executor/adapter.go @@ -556,8 +556,13 @@ func (a *ExecStmt) handlePessimisticDML(ctx context.Context, e Executor) error { } seVars := sctx.GetSessionVars() lockCtx := newLockCtx(seVars, seVars.LockWaitTimeout) + var lockKeyStats *execdetails.LockKeysDetails + ctx = context.WithValue(ctx, execdetails.LockKeysDetailCtxKey, &lockKeyStats) startLocking := time.Now() err = txn.LockKeys(ctx, lockCtx, keys...) + if lockKeyStats != nil { + seVars.StmtCtx.MergeLockKeysExecDetails(lockKeyStats) + } if err == nil { return nil } @@ -795,10 +800,21 @@ func (a *ExecStmt) CloseRecordSet(txnStartTS uint64, lastErr error) { // 3. record execute duration metric. // 4. update the `PrevStmt` in session variable. func (a *ExecStmt) FinishExecuteStmt(txnTS uint64, succ bool, hasMoreResults bool) { + sessVars := a.Ctx.GetSessionVars() + execDetail := sessVars.StmtCtx.GetExecDetails() + // Attach commit/lockKeys runtime stats to executor runtime stats. + if (execDetail.CommitDetail != nil || execDetail.LockKeysDetail != nil) && sessVars.StmtCtx.RuntimeStatsColl != nil { + stats := sessVars.StmtCtx.RuntimeStatsColl.GetRootStats(a.Plan.ID()) + statsWithCommit := &execdetails.RuntimeStatsWithCommit{ + RuntimeStats: stats, + Commit: execDetail.CommitDetail, + LockKeys: execDetail.LockKeysDetail, + } + sessVars.StmtCtx.RuntimeStatsColl.RegisterStats(a.Plan.ID(), statsWithCommit) + } // `LowSlowQuery` and `SummaryStmt` must be called before recording `PrevStmt`. a.LogSlowQuery(txnTS, succ, hasMoreResults) a.SummaryStmt(succ) - sessVars := a.Ctx.GetSessionVars() prevStmt := a.GetTextToLog() if config.RedactLogEnabled() { sessVars.PrevStmt = FormatSQL(prevStmt, nil) @@ -850,16 +866,6 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool, hasMoreResults bool) { stmtDetail = *(stmtDetailRaw.(*execdetails.StmtExecDetails)) } execDetail := sessVars.StmtCtx.GetExecDetails() - - // Attach commit runtime stats to executor runtime stats. - if execDetail.CommitDetail != nil && sessVars.StmtCtx.RuntimeStatsColl != nil { - stats := sessVars.StmtCtx.RuntimeStatsColl.GetRootStats(a.Plan.ID()) - statsWithCommit := &execdetails.RuntimeStatsWithCommit{ - RuntimeStats: stats, - Commit: execDetail.CommitDetail, - } - sessVars.StmtCtx.RuntimeStatsColl.RegisterStats(a.Plan.ID(), statsWithCommit) - } copTaskInfo := sessVars.StmtCtx.CopTasksDetails() statsInfos := plannercore.GetStatsInfo(a.Plan) memMax := sessVars.StmtCtx.MemTracker.MaxConsumed() diff --git a/executor/executor.go b/executor/executor.go index 21f57bf5a6176..d2189079de177 100644 --- a/executor/executor.go +++ b/executor/executor.go @@ -953,7 +953,13 @@ func doLockKeys(ctx context.Context, se sessionctx.Context, lockCtx *kv.LockCtx, if err != nil { return err } - return txn.LockKeys(sessionctx.SetCommitCtx(ctx, se), lockCtx, keys...) + var lockKeyStats *execdetails.LockKeysDetails + ctx = context.WithValue(ctx, execdetails.LockKeysDetailCtxKey, &lockKeyStats) + err = txn.LockKeys(sessionctx.SetCommitCtx(ctx, se), lockCtx, keys...) + if lockKeyStats != nil { + sctx.MergeLockKeysExecDetails(lockKeyStats) + } + return err } // LimitExec represents limit executor diff --git a/executor/executor_test.go b/executor/executor_test.go index 035d7631bfa31..29888cd711ad9 100644 --- a/executor/executor_test.go +++ b/executor/executor_test.go @@ -6017,6 +6017,43 @@ func (s *testSuite) TestIssue19372(c *C) { tk.MustQuery("select (select t2.c_str from t2 where t2.c_str <= t1.c_str and t2.c_int in (1, 2) order by t2.c_str limit 1) x from t1 order by c_int;").Check(testkit.Rows("a", "a", "a")) } +func (s *testSuite) TestCollectDMLRuntimeStats(c *C) { + tk := testkit.NewTestKit(c, s.store) + tk.MustExec("use test") + tk.MustExec("drop table if exists t1") + tk.MustExec("create table t1 (a int, b int, unique index (a))") + + testSQLs := []string{ + "insert ignore into t1 values (5,5);", + "insert into t1 values (5,5) on duplicate key update a=a+1;", + "replace into t1 values (5,6),(6,7)", + "update t1 set a=a+1 where a=6;", + } + + getRootStats := func() string { + info := tk.Se.ShowProcess() + c.Assert(info, NotNil) + p, ok := info.Plan.(plannercore.Plan) + c.Assert(ok, IsTrue) + stats := tk.Se.GetSessionVars().StmtCtx.RuntimeStatsColl.GetRootStats(p.ID()) + return stats.String() + } + for _, sql := range testSQLs { + tk.MustExec(sql) + } + + // Test for lock keys stats. + tk.MustExec("begin pessimistic") + tk.MustExec("update t1 set b=b+1") + c.Assert(getRootStats(), Matches, "time.*lock_keys.*time.* region.* keys.* lock_rpc:.* rpc_count.*") + tk.MustExec("rollback") + + tk.MustExec("begin pessimistic") + tk.MustQuery("select * from t1 for update").Check(testkit.Rows("5 6", "7 7")) + c.Assert(getRootStats(), Matches, "time.*lock_keys.*time.* region.* keys.* lock_rpc:.* rpc_count.*") + tk.MustExec("rollback") +} + func (s *testSuite) TestIssue13758(c *C) { tk := testkit.NewTestKit(c, s.store) tk.MustExec("use test") diff --git a/kv/kv.go b/kv/kv.go index ef91524f480d7..25e2b5f5e9ebe 100644 --- a/kv/kv.go +++ b/kv/kv.go @@ -20,6 +20,7 @@ import ( "github.com/pingcap/tidb/config" "github.com/pingcap/tidb/store/tikv/oracle" + "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/memory" ) @@ -236,6 +237,7 @@ type LockCtx struct { ValuesLock sync.Mutex LockExpired *uint32 CheckKeyExists map[string]struct{} + Stats *execdetails.LockKeysDetails } // ReturnedValue pairs the Value and AlreadyLocked flag for PessimisticLock return values result. diff --git a/sessionctx/stmtctx/stmtctx.go b/sessionctx/stmtctx/stmtctx.go index 2089e48c3588d..a51e57ad2b1b4 100644 --- a/sessionctx/stmtctx/stmtctx.go +++ b/sessionctx/stmtctx/stmtctx.go @@ -497,6 +497,17 @@ func (sc *StatementContext) MergeExecDetails(details *execdetails.ExecDetails, c sc.mu.Unlock() } +// MergeLockKeysExecDetails merges lock keys execution details into self. +func (sc *StatementContext) MergeLockKeysExecDetails(lockKeys *execdetails.LockKeysDetails) { + sc.mu.Lock() + if sc.mu.execDetails.LockKeysDetail == nil { + sc.mu.execDetails.LockKeysDetail = lockKeys + } else { + sc.mu.execDetails.LockKeysDetail.Merge(lockKeys) + } + sc.mu.Unlock() +} + // GetExecDetails gets the execution details for the statement. func (sc *StatementContext) GetExecDetails() execdetails.ExecDetails { var details execdetails.ExecDetails diff --git a/store/tikv/2pc.go b/store/tikv/2pc.go index 79bc67bf1ddb0..3307b24699897 100644 --- a/store/tikv/2pc.go +++ b/store/tikv/2pc.go @@ -555,7 +555,9 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh var batches []batchMutations var sizeFunc = c.keySize - if _, ok := action.(actionPrewrite); ok { + + switch act := action.(type) { + case actionPrewrite: // Do not update regionTxnSize on retries. They are not used when building a PrewriteRequest. if len(bo.errors) == 0 { for _, group := range groups { @@ -564,6 +566,10 @@ func (c *twoPhaseCommitter) doActionOnGroupMutations(bo *Backoffer, action twoPh } sizeFunc = c.keyValueSize atomic.AddInt32(&c.getDetail().PrewriteRegionNum, int32(len(groups))) + case actionPessimisticLock: + if act.LockCtx.Stats != nil { + act.LockCtx.Stats.RegionNum = int32(len(groups)) + } } primaryIdx := -1 @@ -934,7 +940,12 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * time.Sleep(300 * time.Millisecond) return kv.ErrWriteConflict }) + startTime := time.Now() resp, err := c.store.SendReq(bo, req, batch.region, readTimeoutShort) + if action.LockCtx.Stats != nil { + atomic.AddInt64(&action.LockCtx.Stats.LockRPCTime, int64(time.Since(startTime))) + atomic.AddInt64(&action.LockCtx.Stats.LockRPCCount, 1) + } if err != nil { return errors.Trace(err) } @@ -989,7 +1000,11 @@ func (action actionPessimisticLock) handleSingleBatch(c *twoPhaseCommitter, bo * } // Because we already waited on tikv, no need to Backoff here. // tikv default will wait 3s(also the maximum wait value) when lock error occurs + startTime = time.Now() msBeforeTxnExpired, _, err := c.store.lockResolver.ResolveLocks(bo, 0, locks) + if action.LockCtx.Stats != nil { + atomic.AddInt64(&action.LockCtx.Stats.ResolveLockTime, int64(time.Since(startTime))) + } if err != nil { return errors.Trace(err) } diff --git a/store/tikv/txn.go b/store/tikv/txn.go index 2f41dd1cb6d20..efb9edf937e1a 100644 --- a/store/tikv/txn.go +++ b/store/tikv/txn.go @@ -367,6 +367,7 @@ func (txn *tikvTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput // Exclude keys that are already locked. var err error keys := make([][]byte, 0, len(keysInput)) + startTime := time.Now() defer func() { if err == nil { if lockCtx.PessimisticLockWaited != nil { @@ -380,6 +381,14 @@ func (txn *tikvTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput if lockCtx.LockKeysCount != nil { *lockCtx.LockKeysCount += int32(len(keys)) } + if lockCtx.Stats != nil { + lockCtx.Stats.TotalTime = time.Since(startTime) + ctxValue := ctx.Value(execdetails.LockKeysDetailCtxKey) + if ctxValue != nil { + lockKeysDetail := ctxValue.(**execdetails.LockKeysDetails) + *lockKeysDetail = lockCtx.Stats + } + } }() for _, key := range keysInput { // The value of lockedMap is only used by pessimistic transactions. @@ -428,12 +437,21 @@ func (txn *tikvTxn) LockKeys(ctx context.Context, lockCtx *kv.LockCtx, keysInput assignedPrimaryKey = true } + lockCtx.Stats = &execdetails.LockKeysDetails{ + LockKeys: int32(len(keys)), + } bo := NewBackofferWithVars(ctx, pessimisticLockMaxBackoff, txn.vars) txn.committer.forUpdateTS = lockCtx.ForUpdateTS // If the number of keys greater than 1, it can be on different region, // concurrently execute on multiple regions may lead to deadlock. txn.committer.isFirstLock = len(txn.lockKeys) == 0 && len(keys) == 1 err = txn.committer.pessimisticLockMutations(bo, lockCtx, CommitterMutations{keys: keys}) + if bo.totalSleep > 0 { + atomic.AddInt64(&lockCtx.Stats.BackoffTime, int64(bo.totalSleep)*int64(time.Millisecond)) + lockCtx.Stats.Mu.Lock() + lockCtx.Stats.Mu.BackoffTypes = append(lockCtx.Stats.Mu.BackoffTypes, bo.types...) + lockCtx.Stats.Mu.Unlock() + } if lockCtx.Killed != nil { // If the kill signal is received during waiting for pessimisticLock, // pessimisticLockKeys would handle the error but it doesn't reset the flag. diff --git a/util/execdetails/execdetails.go b/util/execdetails/execdetails.go index e2ca74ae553a7..c8e5e245d17ad 100644 --- a/util/execdetails/execdetails.go +++ b/util/execdetails/execdetails.go @@ -28,9 +28,15 @@ import ( ) type commitDetailCtxKeyType struct{} +type lockKeysDetailCtxKeyType struct{} -// CommitDetailCtxKey presents CommitDetail info key in context. -var CommitDetailCtxKey = commitDetailCtxKeyType{} +var ( + // CommitDetailCtxKey presents CommitDetail info key in context. + CommitDetailCtxKey = commitDetailCtxKeyType{} + + // LockKeysDetailCtxKey presents LockKeysDetail info key in context. + LockKeysDetailCtxKey = lockKeysDetailCtxKeyType{} +) // ExecDetails contains execution detail information. type ExecDetails struct { @@ -46,6 +52,7 @@ type ExecDetails struct { TotalKeys int64 ProcessedKeys int64 CommitDetail *CommitDetails + LockKeysDetail *LockKeysDetails } type stmtExecDetailKeyType struct{} @@ -81,6 +88,35 @@ type CommitDetails struct { TxnRetry int } +// LockKeysDetails contains pessimistic lock keys detail information. +type LockKeysDetails struct { + TotalTime time.Duration + RegionNum int32 + LockKeys int32 + ResolveLockTime int64 + BackoffTime int64 + Mu struct { + sync.Mutex + BackoffTypes []fmt.Stringer + } + LockRPCTime int64 + LockRPCCount int64 + RetryCount int +} + +// Merge merges lock keys execution details into self. +func (ld *LockKeysDetails) Merge(lockKey *LockKeysDetails) { + ld.TotalTime += lockKey.TotalTime + ld.RegionNum += lockKey.RegionNum + ld.LockKeys += lockKey.LockKeys + ld.ResolveLockTime += lockKey.ResolveLockTime + ld.BackoffTime += lockKey.BackoffTime + ld.LockRPCTime += lockKey.LockRPCTime + ld.LockRPCCount += ld.LockRPCCount + ld.Mu.BackoffTypes = append(ld.Mu.BackoffTypes, lockKey.Mu.BackoffTypes...) + ld.RetryCount++ +} + const ( // CopTimeStr represents the sum of cop-task time spend in TiDB distSQL. CopTimeStr = "Cop_time" @@ -496,80 +532,130 @@ func (e *RuntimeStatsWithConcurrencyInfo) String() string { // RuntimeStatsWithCommit is the RuntimeStats with commit detail. type RuntimeStatsWithCommit struct { RuntimeStats - Commit *CommitDetails + Commit *CommitDetails + LockKeys *LockKeysDetails } func (e *RuntimeStatsWithCommit) String() string { - var result string + buf := bytes.NewBuffer(make([]byte, 0, 32)) if e.RuntimeStats != nil { - result = e.RuntimeStats.String() - } - if e.Commit == nil { - return result - } - buf := bytes.NewBuffer(make([]byte, 0, len(result)+32)) - buf.WriteString(result) - if e.Commit.PrewriteTime > 0 { - buf.WriteString(", prewrite:") - buf.WriteString(e.Commit.PrewriteTime.String()) - } - if e.Commit.WaitPrewriteBinlogTime > 0 { - buf.WriteString(", wait_prewrite_binlog:") - buf.WriteString(e.Commit.WaitPrewriteBinlogTime.String()) - } - if e.Commit.GetCommitTsTime > 0 { - buf.WriteString(", get_commit_ts:") - buf.WriteString(e.Commit.GetCommitTsTime.String()) - } - if e.Commit.CommitTime > 0 { - buf.WriteString(", commit:") - buf.WriteString(e.Commit.CommitTime.String()) - } - commitBackoffTime := atomic.LoadInt64(&e.Commit.CommitBackoffTime) - if commitBackoffTime > 0 { - buf.WriteString(", commit_backoff: {time: ") - buf.WriteString(time.Duration(commitBackoffTime).String()) - tpMap := make(map[string]struct{}) - tpArray := []string{} - e.Commit.Mu.Lock() - if len(e.Commit.Mu.BackoffTypes) > 0 { - for _, tp := range e.Commit.Mu.BackoffTypes { - tpStr := tp.String() - _, ok := tpMap[tpStr] - if ok { - continue - } - tpMap[tpStr] = struct{}{} - tpArray = append(tpArray, tpStr) + buf.WriteString(e.RuntimeStats.String()) + } + if e.Commit != nil { + buf.WriteString(", commit_txn: {") + if e.Commit.PrewriteTime > 0 { + buf.WriteString("prewrite:") + buf.WriteString(e.Commit.PrewriteTime.String()) + } + if e.Commit.WaitPrewriteBinlogTime > 0 { + buf.WriteString(", wait_prewrite_binlog:") + buf.WriteString(e.Commit.WaitPrewriteBinlogTime.String()) + } + if e.Commit.GetCommitTsTime > 0 { + buf.WriteString(", get_commit_ts:") + buf.WriteString(e.Commit.GetCommitTsTime.String()) + } + if e.Commit.CommitTime > 0 { + buf.WriteString(", commit:") + buf.WriteString(e.Commit.CommitTime.String()) + } + commitBackoffTime := atomic.LoadInt64(&e.Commit.CommitBackoffTime) + if commitBackoffTime > 0 { + buf.WriteString(", backoff: {time: ") + buf.WriteString(time.Duration(commitBackoffTime).String()) + e.Commit.Mu.Lock() + if len(e.Commit.Mu.BackoffTypes) > 0 { + buf.WriteString(", type: ") + buf.WriteString(e.formatBackoff(e.Commit.Mu.BackoffTypes)) } - buf.WriteString(", type: ") - sort.Strings(tpArray) - buf.WriteString(fmt.Sprintf("%v", tpArray)) + e.Commit.Mu.Unlock() + buf.WriteString("}") + } + if e.Commit.ResolveLockTime > 0 { + buf.WriteString(", resolve_lock: ") + buf.WriteString(time.Duration(e.Commit.ResolveLockTime).String()) + } + + prewriteRegionNum := atomic.LoadInt32(&e.Commit.PrewriteRegionNum) + if prewriteRegionNum > 0 { + buf.WriteString(", region_num:") + buf.WriteString(strconv.FormatInt(int64(prewriteRegionNum), 10)) + } + if e.Commit.WriteKeys > 0 { + buf.WriteString(", write_keys:") + buf.WriteString(strconv.FormatInt(int64(e.Commit.WriteKeys), 10)) + } + if e.Commit.WriteSize > 0 { + buf.WriteString(", write_byte:") + buf.WriteString(strconv.FormatInt(int64(e.Commit.WriteSize), 10)) + } + if e.Commit.TxnRetry > 0 { + buf.WriteString(", txn_retry:") + buf.WriteString(strconv.FormatInt(int64(e.Commit.TxnRetry), 10)) } - e.Commit.Mu.Unlock() buf.WriteString("}") } - if e.Commit.ResolveLockTime > 0 { - buf.WriteString(", resolve_lock: ") - buf.WriteString(time.Duration(e.Commit.ResolveLockTime).String()) + if e.LockKeys != nil { + buf.WriteString(", lock_keys: {") + if e.LockKeys.TotalTime > 0 { + buf.WriteString("time:") + buf.WriteString(e.LockKeys.TotalTime.String()) + } + if e.LockKeys.RegionNum > 0 { + buf.WriteString(", region:") + buf.WriteString(strconv.FormatInt(int64(e.LockKeys.RegionNum), 10)) + } + if e.LockKeys.LockKeys > 0 { + buf.WriteString(", keys:") + buf.WriteString(strconv.FormatInt(int64(e.LockKeys.LockKeys), 10)) + } + if e.LockKeys.ResolveLockTime > 0 { + buf.WriteString(", resolve_lock:") + buf.WriteString(time.Duration(e.LockKeys.ResolveLockTime).String()) + } + if e.LockKeys.BackoffTime > 0 { + buf.WriteString(", backoff: {time: ") + buf.WriteString(time.Duration(e.LockKeys.BackoffTime).String()) + e.LockKeys.Mu.Lock() + if len(e.LockKeys.Mu.BackoffTypes) > 0 { + buf.WriteString(", type: ") + buf.WriteString(e.formatBackoff(e.LockKeys.Mu.BackoffTypes)) + } + e.LockKeys.Mu.Unlock() + buf.WriteString("}") + } + if e.LockKeys.LockRPCTime > 0 { + buf.WriteString(", lock_rpc:") + buf.WriteString(time.Duration(e.LockKeys.LockRPCTime).String()) + } + if e.LockKeys.LockRPCCount > 0 { + buf.WriteString(", rpc_count:") + buf.WriteString(strconv.FormatInt(e.LockKeys.LockRPCCount, 10)) + } + if e.LockKeys.RetryCount > 0 { + buf.WriteString(", retry_count:") + buf.WriteString(strconv.FormatInt(int64(e.LockKeys.RetryCount), 10)) + } + buf.WriteString("}") } + return buf.String() +} - prewriteRegionNum := atomic.LoadInt32(&e.Commit.PrewriteRegionNum) - if prewriteRegionNum > 0 { - buf.WriteString(", region_num:") - buf.WriteString(strconv.FormatInt(int64(prewriteRegionNum), 10)) - } - if e.Commit.WriteKeys > 0 { - buf.WriteString(", write_keys:") - buf.WriteString(strconv.FormatInt(int64(e.Commit.WriteKeys), 10)) - } - if e.Commit.WriteSize > 0 { - buf.WriteString(", write_byte:") - buf.WriteString(strconv.FormatInt(int64(e.Commit.WriteSize), 10)) +func (e *RuntimeStatsWithCommit) formatBackoff(backoffTypes []fmt.Stringer) string { + if len(backoffTypes) == 0 { + return "" } - if e.Commit.TxnRetry > 0 { - buf.WriteString(", txn_retry:") - buf.WriteString(strconv.FormatInt(int64(e.Commit.TxnRetry), 10)) + tpMap := make(map[string]struct{}) + tpArray := []string{} + for _, tp := range backoffTypes { + tpStr := tp.String() + _, ok := tpMap[tpStr] + if ok { + continue + } + tpMap[tpStr] = struct{}{} + tpArray = append(tpArray, tpStr) } - return buf.String() + sort.Strings(tpArray) + return fmt.Sprintf("%v", tpArray) } diff --git a/util/execdetails/execdetails_test.go b/util/execdetails/execdetails_test.go index 9ee45d1b070df..a16bcbc66db84 100644 --- a/util/execdetails/execdetails_test.go +++ b/util/execdetails/execdetails_test.go @@ -154,7 +154,39 @@ func TestRuntimeStatsWithCommit(t *testing.T) { RuntimeStats: basicStats, Commit: commitDetail, } - expect := "time:1s, loops:1, prewrite:1s, get_commit_ts:1s, commit:1s, commit_backoff: {time: 1s, type: [backoff1 backoff2]}, resolve_lock: 1s, region_num:5, write_keys:3, write_byte:66, txn_retry:2" + expect := "time:1s, loops:1, commit_txn: {prewrite:1s, get_commit_ts:1s, commit:1s, backoff: {time: 1s, type: [backoff1 backoff2]}, resolve_lock: 1s, region_num:5, write_keys:3, write_byte:66, txn_retry:2}" + if stats.String() != expect { + t.Fatalf("%v != %v", stats.String(), expect) + } + lockDetail := &LockKeysDetails{ + TotalTime: time.Second, + RegionNum: 2, + LockKeys: 10, + ResolveLockTime: int64(time.Second * 2), + 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" + }), + }}, + LockRPCTime: int64(time.Second * 5), + LockRPCCount: 50, + RetryCount: 2, + } + stats = &RuntimeStatsWithCommit{ + RuntimeStats: basicStats, + LockKeys: lockDetail, + } + expect = "time:1s, loops:1, lock_keys: {time:1s, region:2, keys:10, resolve_lock:2s, backoff: {time: 3s, type: [backoff4 backoff5]}, lock_rpc:5s, rpc_count:50, retry_count:2}" if stats.String() != expect { t.Fatalf("%v != %v", stats.String(), expect) }