Skip to content

Commit

Permalink
distsql: fix cop task runtime stats redundant display (pingcap#19674)
Browse files Browse the repository at this point in the history
  • Loading branch information
crazycs520 authored Sep 2, 2020
1 parent bc63f46 commit 5f9301b
Show file tree
Hide file tree
Showing 2 changed files with 46 additions and 0 deletions.
22 changes: 22 additions & 0 deletions distsql/distsql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,9 +27,11 @@ import (
"github.com/pingcap/tidb/sessionctx"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/store/tikv"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tipb/go-tipb"
)
Expand Down Expand Up @@ -156,6 +158,26 @@ func (s *testSuite) TestSelectWithRuntimeStats(c *C) {
c.Assert(err, IsNil)
}

func (s *testSuite) TestSelectResultRuntimeStats(c *C) {
basic := &execdetails.BasicRuntimeStats{}
basic.Record(time.Second, 20)
s1 := &selectResultRuntimeStats{
RuntimeStats: basic,
copRespTime: []time.Duration{time.Second, time.Millisecond},
procKeys: []int64{100, 200},
backoffSleep: map[string]time.Duration{"RegionMiss": time.Millisecond},
totalProcessTime: time.Second,
totalWaitTime: time.Second,
rpcStat: tikv.RegionRequestRuntimeStats{},
}
s2 := *s1
s2.RuntimeStats = s1
expect := "time:1s, loops:1, cop_task: {num: 4, max: 1s, min: 1ms, avg: 500.5ms, p95: 1s, max_proc_keys: 200, p95_proc_keys: 200, tot_proc: 2s, tot_wait: 2s}, backoff{RegionMiss: 2ms}"
c.Assert(s2.String(), Equals, expect)
// Test for idempotence.
c.Assert(s2.String(), Equals, expect)
}

func (s *testSuite) createSelectStreaming(batch, totalRows int, c *C) (*streamResult, []*types.FieldType) {
request, err := (&RequestBuilder{}).SetKeyRanges(nil).
SetDAGRequest(&tipb.DAGRequest{}).
Expand Down
24 changes: 24 additions & 0 deletions distsql/select_result.go
Original file line number Diff line number Diff line change
Expand Up @@ -344,9 +344,33 @@ func (s *selectResultRuntimeStats) mergeCopRuntimeStats(copStats *tikv.CopRuntim
s.rpcStat.Merge(copStats.RegionRequestRuntimeStats)
}

func (s *selectResultRuntimeStats) merge(other *selectResultRuntimeStats) {
s.copRespTime = append(s.copRespTime, other.copRespTime...)
s.procKeys = append(s.procKeys, other.procKeys...)

for k, v := range other.backoffSleep {
s.backoffSleep[k] += v
}
s.totalProcessTime += other.totalProcessTime
s.totalWaitTime += other.totalWaitTime
s.rpcStat.Merge(other.rpcStat)
}

func (s *selectResultRuntimeStats) String() string {
buf := bytes.NewBuffer(nil)
if s.RuntimeStats != nil {
stats, ok := s.RuntimeStats.(*selectResultRuntimeStats)
if ok {
stats.merge(s)
// Clean for idempotence.
s.copRespTime = nil
s.procKeys = nil
s.backoffSleep = nil
s.totalWaitTime = 0
s.totalProcessTime = 0
s.rpcStat = tikv.RegionRequestRuntimeStats{}
return stats.String()
}
buf.WriteString(s.RuntimeStats.String())
}
if len(s.copRespTime) > 0 {
Expand Down

0 comments on commit 5f9301b

Please sign in to comment.