Skip to content

Commit 74c89a1

Browse files
ti-chi-botcrazycs520
authored andcommitted
*: add rpc error stats and refine log (pingcap#52810) (pingcap#52998)
close pingcap#52809 Signed-off-by: crazycs520 <[email protected]>
1 parent e043013 commit 74c89a1

File tree

11 files changed

+62
-47
lines changed

11 files changed

+62
-47
lines changed

go.mod

+2
Original file line numberDiff line numberDiff line change
@@ -318,3 +318,5 @@ replace (
318318
sourcegraph.com/sourcegraph/appdash => github.com/sourcegraph/appdash v0.0.0-20190731080439-ebfcffb1b5c0
319319
sourcegraph.com/sourcegraph/appdash-data => github.com/sourcegraph/appdash-data v0.0.0-20151005221446-73f23eafcf67
320320
)
321+
322+
replace github.com/tikv/client-go/v2 => github.com/crazycs520/client-go/v2 v2.0.0-alpha.0.20240827064637-56b474cf8d7a

go.sum

+2-2
Original file line numberDiff line numberDiff line change
@@ -225,6 +225,8 @@ github.com/coreos/go-systemd/v22 v22.5.0/go.mod h1:Y58oyj3AT4RCenI/lSvhwexgC+NSV
225225
github.com/cpuguy83/go-md2man v1.0.10/go.mod h1:SmD6nW6nTyfqj6ABTjUi3V3JVMnlJmwcJI5acqYI6dE=
226226
github.com/cpuguy83/go-md2man/v2 v2.0.1/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o=
227227
github.com/cpuguy83/go-md2man/v2 v2.0.2/go.mod h1:tgQtvFlXSQOSOSIRvRPT7W67SCa46tRHOmNcaadrF8o=
228+
github.com/crazycs520/client-go/v2 v2.0.0-alpha.0.20240827064637-56b474cf8d7a h1:8BDld/ENT32UlJnWoU8dFzqFneD4kfoct7gRHWjTuQs=
229+
github.com/crazycs520/client-go/v2 v2.0.0-alpha.0.20240827064637-56b474cf8d7a/go.mod h1:37p0ryKaieJbBpVDWnaPi2ZS6UFqkgpsemBLkGX2FvM=
228230
github.com/creack/pty v1.1.9/go.mod h1:oKZEueFk5CKHvIhNR5MUki03XCEU+Q6VDXinZuGJ33E=
229231
github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548 h1:iwZdTE0PVqJCos1vaoKsclOGD3ADKpshg3SRtYBbwso=
230232
github.com/cznic/mathutil v0.0.0-20181122101859-297441e03548/go.mod h1:e6NPNENfs9mPDVNRekM7lKScauxd5kXTr1Mfyig6TDM=
@@ -996,8 +998,6 @@ github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2 h1:mbAskLJ0oJf
996998
github.com/tiancaiamao/appdash v0.0.0-20181126055449-889f96f722a2/go.mod h1:2PfKggNGDuadAa0LElHrByyrz4JPZ9fFx6Gs7nx7ZZU=
997999
github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a h1:J/YdBZ46WKpXsxsW93SG+q0F8KI+yFrcIDT4c/RNoc4=
9981000
github.com/tiancaiamao/gp v0.0.0-20221230034425-4025bc8a4d4a/go.mod h1:h4xBhSNtOeEosLJ4P7JyKXX7Cabg7AVkWCK5gV2vOrM=
999-
github.com/tikv/client-go/v2 v2.0.8-0.20240807051551-73815ad261dc h1:dHXK/ahoO3PhU9oqiYwB4BBNl2PpgJiZODmtWAfxygo=
1000-
github.com/tikv/client-go/v2 v2.0.8-0.20240807051551-73815ad261dc/go.mod h1:37p0ryKaieJbBpVDWnaPi2ZS6UFqkgpsemBLkGX2FvM=
10011001
github.com/tikv/pd/client v0.0.0-20240724132535-fcb34c90790c h1:oZygf/SCdTUhjoHuZRE85EBgK0oA6LjikpWuJqqjM8U=
10021002
github.com/tikv/pd/client v0.0.0-20240724132535-fcb34c90790c/go.mod h1:NW6Af689Jw1FDxjq+WL0nqOdmQ1XT0ly2R1SIKfQuUw=
10031003
github.com/timakin/bodyclose v0.0.0-20230421092635-574207250966 h1:quvGphlmUVU+nhpFa4gg4yJyTRJ13reZMDHrKwYw53M=

pkg/distsql/distsql_test.go

+6-4
Original file line numberDiff line numberDiff line change
@@ -113,7 +113,7 @@ func TestSelectResultRuntimeStats(t *testing.T) {
113113
backoffSleep: map[string]time.Duration{"RegionMiss": time.Millisecond},
114114
totalProcessTime: time.Second,
115115
totalWaitTime: time.Second,
116-
rpcStat: tikv.NewRegionRequestRuntimeStats(),
116+
reqStat: tikv.NewRegionRequestRuntimeStats(),
117117
distSQLConcurrency: 15,
118118
}
119119
s1.copRespTime.Add(execdetails.Duration(time.Second))
@@ -130,13 +130,15 @@ func TestSelectResultRuntimeStats(t *testing.T) {
130130
// Test for idempotence.
131131
require.Equal(t, expect, stats.String())
132132

133-
s1.rpcStat.Stats[tikvrpc.CmdCop] = &tikv.RPCRuntimeStats{
133+
s1.reqStat.RPCStats[tikvrpc.CmdCop] = &tikv.RPCRuntimeStats{
134134
Count: 1,
135135
Consume: int64(time.Second),
136136
}
137+
s1.reqStat.RecordRPCErrorStats("server_is_busy")
138+
s1.reqStat.RecordRPCErrorStats("server_is_busy")
137139
stmtStats.RegisterStats(2, s1)
138140
stats = stmtStats.GetRootStats(2)
139-
expect = "cop_task: {num: 2, max: 1s, min: 1ms, avg: 500.5ms, p95: 1s, max_proc_keys: 200, p95_proc_keys: 200, tot_proc: 1s, tot_wait: 1s, rpc_num: 1, rpc_time: 1s, copr_cache_hit_ratio: 0.00, max_distsql_concurrency: 15}, backoff{RegionMiss: 1ms}"
141+
expect = "cop_task: {num: 2, max: 1s, min: 1ms, avg: 500.5ms, p95: 1s, max_proc_keys: 200, p95_proc_keys: 200, tot_proc: 1s, tot_wait: 1s, copr_cache_hit_ratio: 0.00, max_distsql_concurrency: 15}, rpc_info:{Cop:{num_rpc:1, total_time:1s}, rpc_errors:{server_is_busy:2}}, backoff{RegionMiss: 1ms}"
140142
require.Equal(t, expect, stats.String())
141143
// Test for idempotence.
142144
require.Equal(t, expect, stats.String())
@@ -145,7 +147,7 @@ func TestSelectResultRuntimeStats(t *testing.T) {
145147
backoffSleep: map[string]time.Duration{"RegionMiss": time.Millisecond},
146148
totalProcessTime: time.Second,
147149
totalWaitTime: time.Second,
148-
rpcStat: tikv.NewRegionRequestRuntimeStats(),
150+
reqStat: tikv.NewRegionRequestRuntimeStats(),
149151
}
150152
s1.copRespTime.Add(execdetails.Duration(time.Second))
151153
s1.procKeys.Add(100)

pkg/distsql/select_result.go

+10-19
Original file line numberDiff line numberDiff line change
@@ -46,7 +46,6 @@ import (
4646
"github.com/pingcap/tipb/go-tipb"
4747
tikvmetrics "github.com/tikv/client-go/v2/metrics"
4848
"github.com/tikv/client-go/v2/tikv"
49-
"github.com/tikv/client-go/v2/tikvrpc"
5049
"go.uber.org/zap"
5150
"golang.org/x/exp/maps"
5251
)
@@ -549,7 +548,7 @@ func (r *selectResult) updateCopRuntimeStats(ctx context.Context, copStats *copr
549548
if r.stats == nil {
550549
r.stats = &selectResultRuntimeStats{
551550
backoffSleep: make(map[string]time.Duration),
552-
rpcStat: tikv.NewRegionRequestRuntimeStats(),
551+
reqStat: tikv.NewRegionRequestRuntimeStats(),
553552
distSQLConcurrency: r.distSQLConcurrency,
554553
}
555554
if ci, ok := r.resp.(copr.CopInfo); ok {
@@ -659,7 +658,7 @@ type selectResultRuntimeStats struct {
659658
backoffSleep map[string]time.Duration
660659
totalProcessTime time.Duration
661660
totalWaitTime time.Duration
662-
rpcStat tikv.RegionRequestRuntimeStats
661+
reqStat *tikv.RegionRequestRuntimeStats
663662
distSQLConcurrency int
664663
extraConcurrency int
665664
CoprCacheHitNum int64
@@ -678,7 +677,7 @@ func (s *selectResultRuntimeStats) mergeCopRuntimeStats(copStats *copr.CopRuntim
678677
maps.Copy(s.backoffSleep, copStats.BackoffSleep)
679678
s.totalProcessTime += copStats.TimeDetail.ProcessTime
680679
s.totalWaitTime += copStats.TimeDetail.WaitTime
681-
s.rpcStat.Merge(copStats.RegionRequestRuntimeStats)
680+
s.reqStat.Merge(copStats.ReqStats)
682681
if copStats.CoprCacheHit {
683682
s.CoprCacheHitNum++
684683
}
@@ -689,7 +688,7 @@ func (s *selectResultRuntimeStats) Clone() execdetails.RuntimeStats {
689688
copRespTime: execdetails.Percentile[execdetails.Duration]{},
690689
procKeys: execdetails.Percentile[execdetails.Int64]{},
691690
backoffSleep: make(map[string]time.Duration, len(s.backoffSleep)),
692-
rpcStat: tikv.NewRegionRequestRuntimeStats(),
691+
reqStat: tikv.NewRegionRequestRuntimeStats(),
693692
distSQLConcurrency: s.distSQLConcurrency,
694693
extraConcurrency: s.extraConcurrency,
695694
CoprCacheHitNum: s.CoprCacheHitNum,
@@ -704,7 +703,7 @@ func (s *selectResultRuntimeStats) Clone() execdetails.RuntimeStats {
704703
}
705704
newRs.totalProcessTime += s.totalProcessTime
706705
newRs.totalWaitTime += s.totalWaitTime
707-
maps.Copy(newRs.rpcStat.Stats, s.rpcStat.Stats)
706+
newRs.reqStat = s.reqStat.Clone()
708707
return &newRs
709708
}
710709

@@ -721,7 +720,7 @@ func (s *selectResultRuntimeStats) Merge(rs execdetails.RuntimeStats) {
721720
}
722721
s.totalProcessTime += other.totalProcessTime
723722
s.totalWaitTime += other.totalWaitTime
724-
s.rpcStat.Merge(other.rpcStat)
723+
s.reqStat.Merge(other.reqStat)
725724
s.CoprCacheHitNum += other.CoprCacheHitNum
726725
if other.distSQLConcurrency > s.distSQLConcurrency {
727726
s.distSQLConcurrency = other.distSQLConcurrency
@@ -736,7 +735,7 @@ func (s *selectResultRuntimeStats) Merge(rs execdetails.RuntimeStats) {
736735

737736
func (s *selectResultRuntimeStats) String() string {
738737
buf := bytes.NewBuffer(nil)
739-
rpcStat := s.rpcStat
738+
reqStat := s.reqStat
740739
if s.copRespTime.Size() > 0 {
741740
size := s.copRespTime.Size()
742741
if size == 1 {
@@ -767,15 +766,6 @@ func (s *selectResultRuntimeStats) String() string {
767766
buf.WriteString(execdetails.FormatDuration(s.totalWaitTime))
768767
}
769768
}
770-
copRPC := rpcStat.Stats[tikvrpc.CmdCop]
771-
if copRPC != nil && copRPC.Count > 0 {
772-
rpcStat = rpcStat.Clone()
773-
delete(rpcStat.Stats, tikvrpc.CmdCop)
774-
buf.WriteString(", rpc_num: ")
775-
buf.WriteString(strconv.FormatInt(copRPC.Count, 10))
776-
buf.WriteString(", rpc_time: ")
777-
buf.WriteString(execdetails.FormatDuration(time.Duration(copRPC.Consume)))
778-
}
779769
if config.GetGlobalConfig().TiKVClient.CoprCache.CapacityMB > 0 {
780770
fmt.Fprintf(buf, ", copr_cache_hit_ratio: %v",
781771
strconv.FormatFloat(s.calcCacheHit(), 'f', 2, 64))
@@ -805,10 +795,11 @@ func (s *selectResultRuntimeStats) String() string {
805795
buf.WriteString("}")
806796
}
807797

808-
rpcStatsStr := rpcStat.String()
798+
rpcStatsStr := reqStat.String()
809799
if len(rpcStatsStr) > 0 {
810-
buf.WriteString(", ")
800+
buf.WriteString(", rpc_info:{")
811801
buf.WriteString(rpcStatsStr)
802+
buf.WriteString("}")
812803
}
813804

814805
if len(s.backoffSleep) > 0 {

pkg/executor/distsql_test.go

+8-8
Original file line numberDiff line numberDiff line change
@@ -427,16 +427,16 @@ func TestCoprocessorPagingSize(t *testing.T) {
427427
// Check 'rpc_num' in the execution information
428428
//
429429
// mysql> explain analyze select * from t_paging;
430-
// +--------------------+----------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
431-
// | id |task | execution info |
432-
// +--------------------+----------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
433-
// | TableReader_5 |root | time:7.27ms, loops:2, cop_task: {num: 10, max: 1.57ms, min: 313.3µs, avg: 675.9µs, p95: 1.57ms, tot_proc: 2ms, rpc_num: 10, rpc_time: 6.69ms, copr_cache_hit_ratio: 0.00, distsql_concurrency: 15} |
434-
// | └─TableFullScan_4 |cop[tikv] | tikv_task:{proc max:1.48ms, min:294µs, avg: 629µs, p80:1.21ms, p95:1.48ms, iters:0, tasks:10} |
435-
// +--------------------+----------+------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
430+
// +--------------------+----------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
431+
// | id |task | execution info |
432+
// +--------------------+----------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
433+
// | TableReader_5 |root | time:7.27ms, loops:2, cop_task: {num: 10, max: 1.57ms, min: 313.3µs, avg: 675.9µs, p95: 1.57ms, tot_proc: 2ms, copr_cache_hit_ratio: 0.00, distsql_concurrency: 15}, rpc_info:{Cop:{num_rpc:10, total_time:6.69ms}} |
434+
// | └─TableFullScan_4 |cop[tikv] | tikv_task:{proc max:1.48ms, min:294µs, avg: 629µs, p80:1.21ms, p95:1.48ms, iters:0, tasks:10} |
435+
// +--------------------+----------+-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------+
436436
// 2 rows in set (0.01 sec)
437437

438-
getRPCNumFromExplain := func(rows [][]interface{}) (res uint64) {
439-
re := regexp.MustCompile("rpc_num: ([0-9]+)")
438+
getRPCNumFromExplain := func(rows [][]any) (res uint64) {
439+
re := regexp.MustCompile("num_rpc:([0-9]+)")
440440
for _, row := range rows {
441441
buf := bytes.NewBufferString("")
442442
_, _ = fmt.Fprintf(buf, "%s\n", row)

pkg/executor/executor_failpoint_test.go

+5-5
Original file line numberDiff line numberDiff line change
@@ -275,7 +275,7 @@ func TestCollectCopRuntimeStats(t *testing.T) {
275275
rows := tk.MustQuery("explain analyze select * from t1").Rows()
276276
require.Len(t, rows, 2)
277277
explain := fmt.Sprintf("%v", rows[0])
278-
require.Regexp(t, ".*rpc_num: .*, .*regionMiss:.*", explain)
278+
require.Regexp(t, ".*num_rpc:.*, .*regionMiss:.*", explain)
279279
require.NoError(t, failpoint.Disable("tikvclient/tikvStoreRespResult"))
280280
}
281281

@@ -586,15 +586,15 @@ func TestTiKVClientReadTimeout(t *testing.T) {
586586
rows = tk.MustQuery("explain analyze select /*+ set_var(tikv_client_read_timeout=1) */ * from t where b > 1").Rows()
587587
require.Len(t, rows, 3)
588588
explain = fmt.Sprintf("%v", rows[0])
589-
require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 2.*", explain)
589+
require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .*num_rpc:2.*", explain)
590590

591591
// Test for stale read.
592592
tk.MustExec("set @a=now(6);")
593593
tk.MustExec("set @@tidb_replica_read='closest-replicas';")
594594
rows = tk.MustQuery("explain analyze select /*+ set_var(tikv_client_read_timeout=1) */ * from t as of timestamp(@a) where b > 1").Rows()
595595
require.Len(t, rows, 3)
596596
explain = fmt.Sprintf("%v", rows[0])
597-
require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 2.*", explain)
597+
require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .*num_rpc:2.*", explain)
598598

599599
// Test for tikv_client_read_timeout session variable.
600600
tk.MustExec("set @@tikv_client_read_timeout=1;")
@@ -614,15 +614,15 @@ func TestTiKVClientReadTimeout(t *testing.T) {
614614
rows = tk.MustQuery("explain analyze select * from t where b > 1").Rows()
615615
require.Len(t, rows, 3)
616616
explain = fmt.Sprintf("%v", rows[0])
617-
require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 2.*", explain)
617+
require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .*num_rpc:2.*", explain)
618618

619619
// Test for stale read.
620620
tk.MustExec("set @a=now(6);")
621621
tk.MustExec("set @@tidb_replica_read='closest-replicas';")
622622
rows = tk.MustQuery("explain analyze select * from t as of timestamp(@a) where b > 1").Rows()
623623
require.Len(t, rows, 3)
624624
explain = fmt.Sprintf("%v", rows[0])
625-
require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .* rpc_num: 2.*", explain)
625+
require.Regexp(t, ".*TableReader.* root time:.*, loops:.* cop_task: {num: 1, .*num_rpc:2.*", explain)
626626
}
627627

628628
func TestGetMvccByEncodedKeyRegionError(t *testing.T) {

pkg/executor/explainfor_test.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -74,7 +74,7 @@ func TestExplainFor(t *testing.T) {
7474
buf.WriteString(fmt.Sprintf("%v", v))
7575
}
7676
}
77-
require.Regexp(t, "TableReader_5 10000.00 0 root time:.*, loops:1,( RU:.*,)? cop_task: {num:.*, max:.*, proc_keys:.* rpc_num: 1, rpc_time:.*} data:TableFullScan_4 N/A N/A\n"+
77+
require.Regexp(t, "TableReader_5 10000.00 0 root time:.*, loops:1,( RU:.*,)? cop_task: {num:.*, max:.*, proc_keys:.*num_rpc:1, total_time:.*} data:TableFullScan_4 N/A N/A\n"+
7878
"└─TableFullScan_4 10000.00 0 cop.* table:t1 tikv_task:{time:.*, loops:0} keep order:false, stats:pseudo N/A N/A",
7979
buf.String())
8080
}

pkg/server/conn_test.go

+10
Original file line numberDiff line numberDiff line change
@@ -722,6 +722,16 @@ func TestConnExecutionTimeout(t *testing.T) {
722722
tk.MustQuery("select SLEEP(1);").Check(testkit.Rows("0"))
723723
err := tk.QueryToErr("select * FROM testTable2 WHERE SLEEP(1);")
724724
require.Equal(t, "[executor:3024]Query execution was interrupted, maximum statement execution time exceeded", err.Error())
725+
// Test executor stats when execution time exceeded.
726+
tk.MustExec("set @@tidb_slow_log_threshold=300")
727+
require.NoError(t, failpoint.Enable("github.com/pingcap/tidb/pkg/store/mockstore/unistore/unistoreRPCSlowByInjestSleep", `return(150)`))
728+
err = tk.QueryToErr("select /*+ max_execution_time(600), set_var(tikv_client_read_timeout=100) */ * from testTable2")
729+
require.NoError(t, failpoint.Disable("github.com/pingcap/tidb/pkg/store/mockstore/unistore/unistoreRPCSlowByInjestSleep"))
730+
require.Error(t, err)
731+
require.Equal(t, "[executor:3024]Query execution was interrupted, maximum statement execution time exceeded", err.Error())
732+
planInfo, err := plancodec.DecodePlan(tk.Session().GetSessionVars().StmtCtx.GetEncodedPlan())
733+
require.NoError(t, err)
734+
require.Regexp(t, "TableReader.*cop_task: {num: .*num_rpc:.*, total_time:.*", planInfo)
725735

726736
// Killed because of max execution time, reset Killed to 0.
727737
atomic.CompareAndSwapUint32(&tk.Session().GetSessionVars().Killed, 2, 0)

pkg/store/copr/batch_request_sender.go

+1-1
Original file line numberDiff line numberDiff line change
@@ -76,7 +76,7 @@ func (ss *RegionBatchRequestSender) SendReqToAddr(bo *Backoffer, rpcCtx *tikv.RP
7676
start := time.Now()
7777
resp, err = ss.GetClient().SendRequest(ctx, rpcCtx.Addr, req, timout)
7878
if ss.Stats != nil && ss.enableCollectExecutionInfo {
79-
tikv.RecordRegionRequestRuntimeStats(ss.Stats, req.Type, time.Since(start))
79+
ss.Stats.RecordRPCRuntimeStats(req.Type, time.Since(start))
8080
}
8181
if err != nil {
8282
cancel()

pkg/store/copr/coprocessor.go

+13-3
Original file line numberDiff line numberDiff line change
@@ -1220,7 +1220,7 @@ func (worker *copIteratorWorker) handleTaskOnce(bo *Backoffer, task *copTask, ch
12201220
req.StoreTp = getEndPointType(task.storeType)
12211221
startTime := time.Now()
12221222
if worker.kvclient.Stats == nil {
1223-
worker.kvclient.Stats = make(map[tikvrpc.CmdType]*tikv.RPCRuntimeStats)
1223+
worker.kvclient.Stats = tikv.NewRegionRequestRuntimeStats()
12241224
}
12251225
// set ReadReplicaScope and TxnScope so that req.IsStaleRead will be true when it's a global scope stale read.
12261226
req.ReadReplicaScope = worker.req.ReadReplicaScope
@@ -1305,10 +1305,16 @@ const (
13051305

13061306
func (worker *copIteratorWorker) logTimeCopTask(costTime time.Duration, task *copTask, bo *Backoffer, resp *coprocessor.Response) {
13071307
logStr := fmt.Sprintf("[TIME_COP_PROCESS] resp_time:%s txnStartTS:%d region_id:%d store_addr:%s", costTime, worker.req.StartTs, task.region.GetID(), task.storeAddr)
1308+
if worker.kvclient.Stats != nil {
1309+
logStr += fmt.Sprintf(" stats:%s", worker.kvclient.Stats.String())
1310+
}
13081311
if bo.GetTotalSleep() > minLogBackoffTime {
13091312
backoffTypes := strings.ReplaceAll(fmt.Sprintf("%v", bo.TiKVBackoffer().GetTypes()), " ", ",")
13101313
logStr += fmt.Sprintf(" backoff_ms:%d backoff_types:%s", bo.GetTotalSleep(), backoffTypes)
13111314
}
1315+
if regionErr := resp.GetRegionError(); regionErr != nil {
1316+
logStr += fmt.Sprintf(" region_err:%s", regionErr.String())
1317+
}
13121318
// resp might be nil, but it is safe to call resp.GetXXX here.
13131319
detailV2 := resp.GetExecDetailsV2()
13141320
detail := resp.GetExecDetails()
@@ -1756,7 +1762,11 @@ func (worker *copIteratorWorker) handleCollectExecutionInfo(bo *Backoffer, rpcCt
17561762
if resp.detail == nil {
17571763
resp.detail = new(CopRuntimeStats)
17581764
}
1759-
resp.detail.Stats = worker.kvclient.Stats
1765+
worker.collectCopRuntimeStats(resp.detail, bo, rpcCtx, resp)
1766+
}
1767+
1768+
func (worker *copIteratorWorker) collectCopRuntimeStats(copStats *CopRuntimeStats, bo *Backoffer, rpcCtx *tikv.RPCContext, resp *copResponse) {
1769+
copStats.ReqStats = worker.kvclient.Stats
17601770
backoffTimes := bo.GetBackoffTimes()
17611771
resp.detail.BackoffTime = time.Duration(bo.GetTotalSleep()) * time.Millisecond
17621772
resp.detail.BackoffSleep = make(map[string]time.Duration, len(backoffTimes))
@@ -1796,7 +1806,7 @@ func (worker *copIteratorWorker) handleCollectExecutionInfo(bo *Backoffer, rpcCt
17961806
// CopRuntimeStats contains execution detail information.
17971807
type CopRuntimeStats struct {
17981808
execdetails.ExecDetails
1799-
tikv.RegionRequestRuntimeStats
1809+
ReqStats *tikv.RegionRequestRuntimeStats
18001810

18011811
CoprCacheHit bool
18021812
}

0 commit comments

Comments
 (0)