Skip to content

Commit

Permalink
executor: show max-proc-keys and p95-proc-keys in results of `exp…
Browse files Browse the repository at this point in the history
…lain analyze` (#13692)
  • Loading branch information
qw4990 authored and sre-bot committed Nov 27, 2019
1 parent d811fbb commit e4bdb7b
Show file tree
Hide file tree
Showing 4 changed files with 42 additions and 11 deletions.
8 changes: 5 additions & 3 deletions distsql/select_result.go
Expand Up @@ -27,6 +27,7 @@ import (
"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/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tipb/go-tipb"
Expand Down Expand Up @@ -107,7 +108,7 @@ func (r *selectResult) fetchResp(ctx context.Context) error {
for _, warning := range r.selectResp.Warnings {
sc.AppendWarning(terror.ClassTiKV.New(terror.ErrCode(warning.Code), warning.Msg))
}
r.updateCopRuntimeStats(resultSubset.GetExecDetails().CalleeAddress, resultSubset.RespTime())
r.updateCopRuntimeStats(resultSubset.GetExecDetails(), resultSubset.RespTime())
r.feedback.Update(resultSubset.GetStartKey(), r.selectResp.OutputCounts)
r.partialCount++
sc.MergeExecDetails(resultSubset.GetExecDetails(), nil)
Expand Down Expand Up @@ -206,7 +207,8 @@ func (r *selectResult) readFromChunk(ctx context.Context, chk *chunk.Chunk) erro
return nil
}

func (r *selectResult) updateCopRuntimeStats(callee string, respTime time.Duration) {
func (r *selectResult) updateCopRuntimeStats(detail *execdetails.ExecDetails, respTime time.Duration) {
callee := detail.CalleeAddress
if r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl == nil || callee == "" {
return
}
Expand All @@ -218,7 +220,7 @@ func (r *selectResult) updateCopRuntimeStats(callee string, respTime time.Durati
return
}

r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RecordOneReaderStats(r.rootPlanID.String(), respTime)
r.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RecordOneReaderStats(r.rootPlanID.String(), respTime, detail)
for i, detail := range r.selectResp.GetExecutionSummaries() {
if detail != nil && detail.TimeProcessedNs != nil &&
detail.NumProducedRows != nil && detail.NumIterations != nil {
Expand Down
6 changes: 3 additions & 3 deletions distsql/select_result_test.go
Expand Up @@ -29,7 +29,7 @@ func (s *testSuite) TestUpdateCopRuntimeStats(c *C) {
sr := selectResult{ctx: ctx}
c.Assert(ctx.GetSessionVars().StmtCtx.RuntimeStatsColl, IsNil)
sr.rootPlanID = copPlan{}
sr.updateCopRuntimeStats("a", 0)
sr.updateCopRuntimeStats(&execdetails.ExecDetails{CalleeAddress: "a"}, 0)

ctx.GetSessionVars().StmtCtx.RuntimeStatsColl = execdetails.NewRuntimeStatsColl()
t := uint64(1)
Expand All @@ -39,13 +39,13 @@ func (s *testSuite) TestUpdateCopRuntimeStats(c *C) {
},
}
c.Assert(len(sr.selectResp.GetExecutionSummaries()) != len(sr.copPlanIDs), IsTrue)
sr.updateCopRuntimeStats("callee", 0)
sr.updateCopRuntimeStats(&execdetails.ExecDetails{CalleeAddress: "callee"}, 0)
c.Assert(ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.ExistsCopStats("callee"), IsFalse)

sr.copPlanIDs = []fmt.Stringer{copPlan{}}
c.Assert(ctx.GetSessionVars().StmtCtx.RuntimeStatsColl, NotNil)
c.Assert(len(sr.selectResp.GetExecutionSummaries()), Equals, len(sr.copPlanIDs))
sr.updateCopRuntimeStats("callee", 0)
sr.updateCopRuntimeStats(&execdetails.ExecDetails{CalleeAddress: "callee"}, 0)
c.Assert(ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.GetCopStats("callee").String(), Equals, "time:1ns, loops:1, rows:1")
}

Expand Down
18 changes: 13 additions & 5 deletions util/execdetails/execdetails.go
Expand Up @@ -280,13 +280,15 @@ type ReaderRuntimeStats struct {
sync.Mutex

copRespTime []time.Duration
procKeys []int64
}

// recordOneCopTask record once cop response time to update maxcopRespTime
func (rrs *ReaderRuntimeStats) recordOneCopTask(t time.Duration) {
func (rrs *ReaderRuntimeStats) recordOneCopTask(t time.Duration, detail *ExecDetails) {
rrs.Lock()
defer rrs.Unlock()
rrs.copRespTime = append(rrs.copRespTime, t)
rrs.procKeys = append(rrs.procKeys, detail.ProcessedKeys)
}

func (rrs *ReaderRuntimeStats) String() string {
Expand All @@ -295,7 +297,7 @@ func (rrs *ReaderRuntimeStats) String() string {
return ""
}
if size == 1 {
return fmt.Sprintf("rpc num:1, time:%v", rrs.copRespTime[0])
return fmt.Sprintf("rpc num: 1, rpc time:%v, proc keys:%v", rrs.copRespTime[0], rrs.procKeys[0])
}
sort.Slice(rrs.copRespTime, func(i, j int) bool {
return rrs.copRespTime[i] < rrs.copRespTime[j]
Expand All @@ -307,7 +309,13 @@ func (rrs *ReaderRuntimeStats) String() string {
sum += float64(t)
}
vAvg := time.Duration(sum / float64(size))
return fmt.Sprintf("rpc num:%v, max:%v, min:%v, avg:%v, p80:%v, p95:%v", size, vMax, vMin, vAvg, vP80, vP95)

sort.Slice(rrs.procKeys, func(i, j int) bool {
return rrs.procKeys[i] < rrs.procKeys[j]
})
keyMax := rrs.procKeys[size-1]
keyP95 := rrs.procKeys[size*19/20]
return fmt.Sprintf("rpc num: %v, rpc max:%v, min:%v, avg:%v, p80:%v, p95:%v, proc keys max:%v, p95:%v", size, vMax, vMin, vAvg, vP80, vP95, keyMax, keyP95)
}

// RuntimeStatsColl collects executors's execution info.
Expand Down Expand Up @@ -365,9 +373,9 @@ func (e *RuntimeStatsColl) RecordOneCopTask(planID, address string, summary *tip
}

// RecordOneReaderStats records a specific stats for TableReader, IndexReader and IndexLookupReader.
func (e *RuntimeStatsColl) RecordOneReaderStats(planID string, copRespTime time.Duration) {
func (e *RuntimeStatsColl) RecordOneReaderStats(planID string, copRespTime time.Duration, detail *ExecDetails) {
readerStats := e.GetReaderStats(planID)
readerStats.recordOneCopTask(copRespTime)
readerStats.recordOneCopTask(copRespTime, detail)
}

// ExistsRootStats checks if the planID exists in the rootStats collection.
Expand Down
21 changes: 21 additions & 0 deletions util/execdetails/execdetails_test.go
Expand Up @@ -110,3 +110,24 @@ func TestCopRuntimeStats(t *testing.T) {
t.Fatal("table_reader not exists")
}
}

func TestReaderStats(t *testing.T) {
r := new(ReaderRuntimeStats)
if r.String() != "" {
t.Fatal()
}

r.procKeys = append(r.procKeys, 100)
r.copRespTime = append(r.copRespTime, time.Millisecond*100)
if r.String() != "rpc num: 1, rpc time:100ms, proc keys:100" {
t.Fatal()
}

for i := 0; i < 100; i++ {
r.procKeys = append(r.procKeys, int64(i))
r.copRespTime = append(r.copRespTime, time.Millisecond*time.Duration(i))
}
if r.String() != "rpc num: 101, rpc max:100ms, min:0s, avg:50ms, p80:80ms, p95:95ms, proc keys max:100, p95:95" {
t.Fatal()
}
}

0 comments on commit e4bdb7b

Please sign in to comment.