Skip to content

Commit

Permalink
*: refine runtime stats display and tiny bug fix for metrics (pingcap…
Browse files Browse the repository at this point in the history
…#21022)

Signed-off-by: crazycs520 <[email protected]>
  • Loading branch information
crazycs520 authored Nov 26, 2020
1 parent 051dc83 commit 32d19b7
Show file tree
Hide file tree
Showing 25 changed files with 401 additions and 122 deletions.
6 changes: 3 additions & 3 deletions bindinfo/bind_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1682,9 +1682,9 @@ func (s *testSuite) TestIssue19836(c *C) {
explainResult := testkit.Rows(
"Limit_8 2.00 0 root time:0s, loops:0 offset:1, count:2 N/A N/A",
"└─TableReader_14 3.00 0 root time:0s, loops:0 data:Limit_13 N/A N/A",
" └─Limit_13 3.00 0 cop[tikv] time:0ns, loops:0 offset:0, count:3 N/A N/A",
" └─Selection_12 3.00 0 cop[tikv] time:0ns, loops:0 eq(test.t.a, 40) N/A N/A",
" └─TableFullScan_11 3000.00 0 cop[tikv] table:t time:0ns, loops:0 keep order:false, stats:pseudo N/A N/A",
" └─Limit_13 3.00 0 cop[tikv] offset:0, count:3 N/A N/A",
" └─Selection_12 3.00 0 cop[tikv] eq(test.t.a, 40) N/A N/A",
" └─TableFullScan_11 3000.00 0 cop[tikv] table:t keep order:false, stats:pseudo N/A N/A",
)
tk.MustQuery("explain for connection " + strconv.FormatUint(tk.Se.ShowProcess().ID, 10)).Check(explainResult)
}
23 changes: 15 additions & 8 deletions distsql/select_result.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ import (

"github.com/pingcap/errors"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/config"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/metrics"
Expand Down Expand Up @@ -407,7 +408,7 @@ func (s *selectResultRuntimeStats) String() string {
if len(s.copRespTime) > 0 {
size := len(s.copRespTime)
if size == 1 {
buf.WriteString(fmt.Sprintf("cop_task: {num: 1, max:%v, proc_keys: %v", s.copRespTime[0], s.procKeys[0]))
buf.WriteString(fmt.Sprintf("cop_task: {num: 1, max: %v, proc_keys: %v", execdetails.FormatDuration(s.copRespTime[0]), s.procKeys[0]))
} else {
sort.Slice(s.copRespTime, func(i, j int) bool {
return s.copRespTime[i] < s.copRespTime[j]
Expand All @@ -425,7 +426,9 @@ func (s *selectResultRuntimeStats) String() string {
})
keyMax := s.procKeys[size-1]
keyP95 := s.procKeys[size*19/20]
buf.WriteString(fmt.Sprintf("cop_task: {num: %v, max: %v, min: %v, avg: %v, p95: %v", size, vMax, vMin, vAvg, vP95))
buf.WriteString(fmt.Sprintf("cop_task: {num: %v, max: %v, min: %v, avg: %v, p95: %v", size,
execdetails.FormatDuration(vMax), execdetails.FormatDuration(vMin),
execdetails.FormatDuration(vAvg), execdetails.FormatDuration(vP95)))
if keyMax > 0 {
buf.WriteString(", max_proc_keys: ")
buf.WriteString(strconv.FormatInt(keyMax, 10))
Expand All @@ -434,10 +437,10 @@ func (s *selectResultRuntimeStats) String() string {
}
if s.totalProcessTime > 0 {
buf.WriteString(", tot_proc: ")
buf.WriteString(s.totalProcessTime.String())
buf.WriteString(execdetails.FormatDuration(s.totalProcessTime))
if s.totalWaitTime > 0 {
buf.WriteString(", tot_wait: ")
buf.WriteString(s.totalWaitTime.String())
buf.WriteString(execdetails.FormatDuration(s.totalWaitTime))
}
}
}
Expand All @@ -448,10 +451,14 @@ func (s *selectResultRuntimeStats) String() string {
buf.WriteString(", rpc_num: ")
buf.WriteString(strconv.FormatInt(copRPC.Count, 10))
buf.WriteString(", rpc_time: ")
buf.WriteString(time.Duration(copRPC.Consume).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(copRPC.Consume)))
}
if config.GetGlobalConfig().TiKVClient.CoprCache.Enable {
buf.WriteString(fmt.Sprintf(", copr_cache_hit_ratio: %v",
strconv.FormatFloat(float64(s.CoprCacheHitNum)/float64(len(s.copRespTime)), 'f', 2, 64)))
} else {
buf.WriteString(", copr_cache: disabled")
}
buf.WriteString(fmt.Sprintf(", copr_cache_hit_ratio: %v",
strconv.FormatFloat(float64(s.CoprCacheHitNum)/float64(len(s.copRespTime)), 'f', 2, 64)))
buf.WriteString("}")
}

Expand All @@ -469,7 +476,7 @@ func (s *selectResultRuntimeStats) String() string {
buf.WriteString(", ")
}
idx++
buf.WriteString(fmt.Sprintf("%s: %s", k, d.String()))
buf.WriteString(fmt.Sprintf("%s: %s", k, execdetails.FormatDuration(d)))
}
buf.WriteString("}")
}
Expand Down
4 changes: 2 additions & 2 deletions executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -1010,13 +1010,13 @@ func (e *IndexLookUpRunTimeStats) String() string {
tableTaskNum := atomic.LoadInt64(&e.TableTaskNum)
concurrency := e.Concurrency
if indexScan != 0 {
buf.WriteString(fmt.Sprintf("index_task:%s", time.Duration(indexScan)))
buf.WriteString(fmt.Sprintf("index_task: %s", execdetails.FormatDuration(time.Duration(indexScan))))
}
if tableScan != 0 {
if buf.Len() > 0 {
buf.WriteByte(',')
}
buf.WriteString(fmt.Sprintf(" table_task:{num:%d, concurrency:%d, time:%s}", tableTaskNum, concurrency, time.Duration(tableScan)))
buf.WriteString(fmt.Sprintf(" table_task: {num: %d, concurrency: %d, time: %s}", tableTaskNum, concurrency, execdetails.FormatDuration(time.Duration(tableScan))))
}
return buf.String()
}
Expand Down
4 changes: 2 additions & 2 deletions executor/distsql_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -264,8 +264,8 @@ func (s *testSuite3) TestIndexLookUpStats(c *C) {
TableTaskNum: 2,
Concurrency: 1,
}
c.Assert(stats.String(), Equals, "index_task:2s, table_task:{num:2, concurrency:1, time:2s}")
c.Assert(stats.String(), Equals, "index_task: 2s, table_task: {num: 2, concurrency: 1, time: 2s}")
c.Assert(stats.String(), Equals, stats.Clone().String())
stats.Merge(stats.Clone())
c.Assert(stats.String(), Equals, "index_task:4s, table_task:{num:4, concurrency:2, time:4s}")
c.Assert(stats.String(), Equals, "index_task: 4s, table_task: {num: 4, concurrency: 2, time: 4s}")
}
48 changes: 28 additions & 20 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -6533,6 +6533,23 @@ func (s *testSerialSuite1) TestCollectCopRuntimeStats(c *C) {
c.Assert(failpoint.Disable("github.com/pingcap/tidb/store/tikv/tikvStoreRespResult"), IsNil)
}

func (s *testSerialSuite1) TestIndexLookupRuntimeStats(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, index(a))")
tk.MustExec("insert into t1 values (1,2),(2,3),(3,4)")
sql := "explain analyze select * from t1 use index(a) where a > 1;"
rows := tk.MustQuery(sql).Rows()
c.Assert(len(rows), Equals, 3)
explain := fmt.Sprintf("%v", rows[0])
c.Assert(explain, Matches, ".*time:.*loops:.*index_task:.*table_task: {num.*concurrency.*time.*}.*")
indexExplain := fmt.Sprintf("%v", rows[1])
tableExplain := fmt.Sprintf("%v", rows[2])
c.Assert(indexExplain, Matches, ".*time:.*loops:.*cop_task:.*")
c.Assert(tableExplain, Matches, ".*time:.*loops:.*cop_task:.*")
}

func (s *testSerialSuite1) TestHashAggRuntimeStats(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test;")
Expand Down Expand Up @@ -6571,23 +6588,6 @@ func (s *testSerialSuite1) TestIndexMergeRuntimeStats(c *C) {
tk.MustQuery(sql).Check(testkit.Rows("1 1 1 1 1", "5 5 5 5 5"))
}

func (s *testSerialSuite1) TestIndexlookupRuntimeStats(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, index(a))")
tk.MustExec("insert into t1 values (1,2),(2,3),(3,4)")
sql := "explain analyze select * from t1 use index(a) where a > 1;"
rows := tk.MustQuery(sql).Rows()
c.Assert(len(rows), Equals, 3)
explain := fmt.Sprintf("%v", rows[0])
c.Assert(explain, Matches, ".*time:.*loops:.*index_task:.*table_task:{num.*concurrency.*time.*}.*")
indexExplain := fmt.Sprintf("%v", rows[1])
tableExplain := fmt.Sprintf("%v", rows[2])
c.Assert(indexExplain, Matches, ".*time:.*loops:.*cop_task:.*")
c.Assert(tableExplain, Matches, ".*time:.*loops:.*cop_task:.*")
}

func (s *testSuite) TestCollectDMLRuntimeStats(c *C) {
tk := testkit.NewTestKit(c, s.store)
tk.MustExec("use test")
Expand Down Expand Up @@ -6627,12 +6627,12 @@ func (s *testSuite) TestCollectDMLRuntimeStats(c *C) {

tk.MustExec("begin pessimistic")
tk.MustExec("insert ignore into t1 values (9,9)")
c.Assert(getRootStats(), Matches, "time:.*, loops:.*, prepare:.*, check_insert:{total_time:.*, mem_insert_time:.*, prefetch:.*, rpc:{BatchGet:{num_rpc:.*, total_time:.*}}}.*")
c.Assert(getRootStats(), Matches, "time:.*, loops:.*, prepare:.*, check_insert: {total_time:.*, mem_insert_time:.*, prefetch:.*, rpc:{BatchGet:{num_rpc:.*, total_time:.*}}}.*")
tk.MustExec("rollback")

tk.MustExec("begin pessimistic")
tk.MustExec("insert into t1 values (10,10) on duplicate key update a=a+1")
c.Assert(getRootStats(), Matches, "time:.*, loops:.*, prepare:.*, check_insert:{total_time:.*, mem_insert_time:.*, prefetch:.*, rpc:{BatchGet:{num_rpc:.*, total_time:.*}.*")
c.Assert(getRootStats(), Matches, "time:.*, loops:.*, prepare:.*, check_insert: {total_time:.*, mem_insert_time:.*, prefetch:.*, rpc:{BatchGet:{num_rpc:.*, total_time:.*}.*")
tk.MustExec("rollback")

tk.MustExec("begin pessimistic")
Expand All @@ -6642,7 +6642,7 @@ func (s *testSuite) TestCollectDMLRuntimeStats(c *C) {

tk.MustExec("begin pessimistic")
tk.MustExec("insert ignore into t1 values(11,11) on duplicate key update `a`=`a`+1")
c.Assert(getRootStats(), Matches, "time:.*, loops:.*, prepare:.*, check_insert:{total_time:.*, mem_insert_time:.*, prefetch:.*, rpc:.*}")
c.Assert(getRootStats(), Matches, "time:.*, loops:.*, prepare:.*, check_insert: {total_time:.*, mem_insert_time:.*, prefetch:.*, rpc:.*}")
tk.MustExec("rollback")

tk.MustExec("begin pessimistic")
Expand Down Expand Up @@ -6725,6 +6725,14 @@ func (s *testCoprCache) TestIntegrationCopCache(c *C) {
hitRatio, err := strconv.ParseFloat(rows[0][5].(string)[hitRatioIdx:hitRatioIdx+4], 64)
c.Assert(err, IsNil)
c.Assert(hitRatio > 0, Equals, true)

// Test for cop cache disabled.
cfg := config.NewConfig()
cfg.TiKVClient.CoprCache.Enable = false
config.StoreGlobalConfig(cfg)
rows = tk.MustQuery("explain analyze select * from t where t.a < 10").Rows()
c.Assert(rows[0][2], Equals, "9")
c.Assert(strings.Contains(rows[0][5].(string), "copr_cache: disabled"), Equals, true)
}

func (s *testSerialSuite) TestCoprocessorOOMTicase(c *C) {
Expand Down
4 changes: 2 additions & 2 deletions executor/explainfor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -110,8 +110,8 @@ func (s *testSerialSuite) TestExplainFor(c *C) {
}
}
c.Assert(buf.String(), Matches, ""+
"TableReader_5 10000.00 0 root time:.*, loops:1, cop_task: {num:.*, max:.*, proc_keys: 0, rpc_num: 1, rpc_time:.*, copr_cache_hit_ratio: 0.00} data:TableFullScan_4 N/A N/A\n"+
"└─TableFullScan_4 10000.00 0 cop.* table:t1 time:.*, loops:0, tikv_task:{time:.*, loops:0} keep order:false, stats:pseudo N/A N/A")
"TableReader_5 10000.00 0 root time:.*, loops:1, cop_task: {num:.*, max:.*, proc_keys: 0, rpc_num: 1, rpc_time:.*} data:TableFullScan_4 N/A N/A\n"+
"└─TableFullScan_4 10000.00 0 cop.* table:t1 tikv_task:{time:.*, loops:0} keep order:false, stats:pseudo N/A N/A")
}
tkRoot.MustQuery("select * from t1;")
check()
Expand Down
3 changes: 2 additions & 1 deletion executor/hash_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@ import (
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/codec"
"github.com/pingcap/tidb/util/disk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/memory"
)

Expand Down Expand Up @@ -82,7 +83,7 @@ type hashStatistic struct {
}

func (s *hashStatistic) String() string {
return fmt.Sprintf("probe_collision:%v, build:%v", s.probeCollision, s.buildTableElapse)
return fmt.Sprintf("probe_collision:%v, build:%v", s.probeCollision, execdetails.FormatDuration(s.buildTableElapse))
}

// hashRowContainer handles the rows and the hash map of a table.
Expand Down
12 changes: 6 additions & 6 deletions executor/index_lookup_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -742,7 +742,7 @@ func (e *indexLookUpJoinRuntimeStats) String() string {
buf := bytes.NewBuffer(make([]byte, 0, 16))
if e.innerWorker.totalTime > 0 {
buf.WriteString("inner:{total:")
buf.WriteString(time.Duration(e.innerWorker.totalTime).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(e.innerWorker.totalTime)))
buf.WriteString(", concurrency:")
if e.concurrency > 0 {
buf.WriteString(strconv.Itoa(e.concurrency))
Expand All @@ -752,20 +752,20 @@ func (e *indexLookUpJoinRuntimeStats) String() string {
buf.WriteString(", task:")
buf.WriteString(strconv.FormatInt(e.innerWorker.task, 10))
buf.WriteString(", construct:")
buf.WriteString(time.Duration(e.innerWorker.construct).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(e.innerWorker.construct)))
buf.WriteString(", fetch:")
buf.WriteString(time.Duration(e.innerWorker.fetch).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(e.innerWorker.fetch)))
buf.WriteString(", build:")
buf.WriteString(time.Duration(e.innerWorker.build).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(e.innerWorker.build)))
if e.innerWorker.join > 0 {
buf.WriteString(", join:")
buf.WriteString(time.Duration(e.innerWorker.join).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(e.innerWorker.join)))
}
buf.WriteString("}")
}
if e.probe > 0 {
buf.WriteString(", probe:")
buf.WriteString(time.Duration(e.probe).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(e.probe)))
}
return buf.String()
}
Expand Down
11 changes: 7 additions & 4 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -1047,22 +1047,25 @@ func (e *InsertRuntimeStat) String() string {
if e.CheckInsertTime == 0 {
// For replace statement.
if e.Prefetch > 0 && e.SnapshotRuntimeStats != nil {
return fmt.Sprintf("prefetch: %v, rpc:{%v}", e.Prefetch, e.SnapshotRuntimeStats.String())
return fmt.Sprintf("prefetch: %v, rpc:{%v}", execdetails.FormatDuration(e.Prefetch), e.SnapshotRuntimeStats.String())
}
return ""
}
buf := bytes.NewBuffer(make([]byte, 0, 32))
buf.WriteString(fmt.Sprintf("prepare:%v, ", time.Duration(e.BasicRuntimeStats.GetTime())-e.CheckInsertTime))
buf.WriteString(fmt.Sprintf("prepare:%v, ", execdetails.FormatDuration(time.Duration(e.BasicRuntimeStats.GetTime())-e.CheckInsertTime)))
if e.Prefetch > 0 {
buf.WriteString(fmt.Sprintf("check_insert:{total_time:%v, mem_insert_time:%v, prefetch:%v", e.CheckInsertTime, e.CheckInsertTime-e.Prefetch, e.Prefetch))
buf.WriteString(fmt.Sprintf("check_insert: {total_time: %v, mem_insert_time: %v, prefetch: %v",
execdetails.FormatDuration(e.CheckInsertTime),
execdetails.FormatDuration(e.CheckInsertTime-e.Prefetch),
execdetails.FormatDuration(e.Prefetch)))
if e.SnapshotRuntimeStats != nil {
if rpc := e.SnapshotRuntimeStats.String(); len(rpc) > 0 {
buf.WriteString(fmt.Sprintf(", rpc:{%s}", rpc))
}
}
buf.WriteString("}")
} else {
buf.WriteString(fmt.Sprintf("insert:%v", e.CheckInsertTime))
buf.WriteString(fmt.Sprintf("insert:%v", execdetails.FormatDuration(e.CheckInsertTime)))
}
return buf.String()
}
Expand Down
4 changes: 2 additions & 2 deletions executor/insert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1428,10 +1428,10 @@ func (s *testSuite10) TestInsertRuntimeStat(c *C) {
Prefetch: 1 * time.Second,
}
stats.BasicRuntimeStats.Record(5*time.Second, 1)
c.Assert(stats.String(), Equals, "prepare:3s, check_insert:{total_time:2s, mem_insert_time:1s, prefetch:1s}")
c.Assert(stats.String(), Equals, "prepare:3s, check_insert: {total_time: 2s, mem_insert_time: 1s, prefetch: 1s}")
c.Assert(stats.String(), Equals, stats.Clone().String())
stats.Merge(stats.Clone())
c.Assert(stats.String(), Equals, "prepare:6s, check_insert:{total_time:4s, mem_insert_time:2s, prefetch:2s}")
c.Assert(stats.String(), Equals, "prepare:6s, check_insert: {total_time: 4s, mem_insert_time: 2s, prefetch: 2s}")
}

func (s *testSerialSuite) TestDuplicateEntryMessage(c *C) {
Expand Down
14 changes: 7 additions & 7 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -1129,24 +1129,24 @@ func (e *hashJoinRuntimeStats) String() string {
buf := bytes.NewBuffer(make([]byte, 0, 128))
if e.fetchAndBuildHashTable > 0 {
buf.WriteString("build_hash_table:{total:")
buf.WriteString(e.fetchAndBuildHashTable.String())
buf.WriteString(execdetails.FormatDuration(e.fetchAndBuildHashTable))
buf.WriteString(", fetch:")
buf.WriteString((e.fetchAndBuildHashTable - e.hashStat.buildTableElapse).String())
buf.WriteString(execdetails.FormatDuration((e.fetchAndBuildHashTable - e.hashStat.buildTableElapse)))
buf.WriteString(", build:")
buf.WriteString(e.hashStat.buildTableElapse.String())
buf.WriteString(execdetails.FormatDuration(e.hashStat.buildTableElapse))
buf.WriteString("}")
}
if e.probe > 0 {
buf.WriteString(", probe:{concurrency:")
buf.WriteString(strconv.Itoa(e.concurrent))
buf.WriteString(", total:")
buf.WriteString(time.Duration(e.fetchAndProbe).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe)))
buf.WriteString(", max:")
buf.WriteString(time.Duration(atomic.LoadInt64(&e.maxFetchAndProbe)).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(atomic.LoadInt64(&e.maxFetchAndProbe))))
buf.WriteString(", probe:")
buf.WriteString(time.Duration(e.probe).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(e.probe)))
buf.WriteString(", fetch:")
buf.WriteString(time.Duration(e.fetchAndProbe - e.probe).String())
buf.WriteString(execdetails.FormatDuration(time.Duration(e.fetchAndProbe - e.probe)))
if e.hashStat.probeCollision > 0 {
buf.WriteString(", probe_collision:")
buf.WriteString(strconv.Itoa(e.hashStat.probeCollision))
Expand Down
5 changes: 3 additions & 2 deletions executor/slow_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -944,8 +944,9 @@ type slowQueryRuntimeStats struct {
// String implements the RuntimeStats interface.
func (s *slowQueryRuntimeStats) String() string {
return fmt.Sprintf("initialize: %s, read_file: %s, parse_log: {time:%s, concurrency:%v}, total_file: %v, read_file: %v, read_size: %s",
s.initialize, s.readFile, time.Duration(s.parseLog), s.concurrent,
s.totalFileNum, s.readFileNum, memory.BytesToString(s.readFileSize))
execdetails.FormatDuration(s.initialize), execdetails.FormatDuration(s.readFile),
execdetails.FormatDuration(time.Duration(s.parseLog)), s.concurrent,
s.totalFileNum, s.readFileNum, memory.FormatBytes(s.readFileSize))
}

// Merge implements the RuntimeStats interface.
Expand Down
2 changes: 1 addition & 1 deletion infoschema/metric_table_def.go
Original file line number Diff line number Diff line change
Expand Up @@ -1843,7 +1843,7 @@ var MetricTableMap = map[string]MetricTableDef{
Comment: "The flow rate of compaction operations per type",
},
"tikv_compaction_pending_bytes": {
PromQL: `sum(rate(tikv_engine_pending_compaction_bytes{$LABEL_CONDITIONS}[$RANGE_DURATION])) by (cf,instance,db)`,
PromQL: `tikv_engine_pending_compaction_bytes{$LABEL_CONDITIONS}`,
Labels: []string{"instance", "cf", "db"},
Comment: "The pending bytes to be compacted",
},
Expand Down
2 changes: 1 addition & 1 deletion planner/core/cbo_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -100,7 +100,7 @@ func (s *testAnalyzeSuite) TestExplainAnalyze(c *C) {
c.Assert(strings.Contains(execInfo, "time"), Equals, true)
c.Assert(strings.Contains(execInfo, "loops"), Equals, true)
if strings.Contains(row[0].(string), "Reader") || strings.Contains(row[0].(string), "IndexLookUp") {
c.Assert(strings.Contains(execInfo, "copr_cache_hit_ratio"), Equals, true)
c.Assert(strings.Contains(execInfo, "cop_task"), Equals, true)
}
}
}
Expand Down
Loading

0 comments on commit 32d19b7

Please sign in to comment.