Skip to content

Commit

Permalink
executor: show memory consumption in slow query log (#10246)
Browse files Browse the repository at this point in the history
  • Loading branch information
qw4990 authored and Zhang Jian committed Apr 25, 2019
1 parent 4a13c71 commit dd64171
Show file tree
Hide file tree
Showing 5 changed files with 61 additions and 5 deletions.
5 changes: 3 additions & 2 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -404,12 +404,13 @@ func (a *ExecStmt) LogSlowQuery(txnTS uint64, succ bool) {
execDetail := sessVars.StmtCtx.GetExecDetails()
copTaskInfo := sessVars.StmtCtx.CopTasksDetails()
statsInfos := a.getStatsInfo()
memMax := sessVars.StmtCtx.MemTracker.MaxConsumed()
if costTime < threshold {
_, digest := sessVars.StmtCtx.SQLDigest()
logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, sql))
logutil.SlowQueryLogger.Debug(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, memMax, sql))
} else {
_, digest := sessVars.StmtCtx.SQLDigest()
logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, sql))
logutil.SlowQueryLogger.Warn(sessVars.SlowLogFormat(txnTS, costTime, execDetail, indexIDs, digest, statsInfos, copTaskInfo, memMax, sql))
metrics.TotalQueryProcHistogram.Observe(costTime.Seconds())
metrics.TotalCopProcHistogram.Observe(execDetail.ProcessTime.Seconds())
metrics.TotalCopWaitHistogram.Observe(execDetail.WaitTime.Seconds())
Expand Down
12 changes: 10 additions & 2 deletions sessionctx/variable/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -790,6 +790,8 @@ const (
SlowLogCopProcessStr = "Cop_process"
// SlowLogCopWaitStr includes some useful information about cop-tasks' wait time.
SlowLogCopWaitStr = "Cop_wait"
// SlowLogMemMax is the max number bytes of memory used in this statement.
SlowLogMemMax = "Mem_max"
)

// SlowLogFormat uses for formatting slow log.
Expand All @@ -805,10 +807,13 @@ const (
// # Is_internal: false
// # Digest: 42a1c8aae6f133e934d4bf0147491709a8812ea05ff8819ec522780fe657b772
// # Stats: t1:1,t2:2
// # Cop_tasks:
// # Num_cop_tasks: 10
// # Cop_process: Avg_time: 1s P90_time: 2s Max_time: 3s Max_addr: 10.6.131.78
// # Cop_wait: Avg_time: 10ms P90_time: 20ms Max_time: 30ms Max_Addr: 10.6.131.79
// # Memory_max: 4096
// select * from t_slim;
func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDetail execdetails.ExecDetails, indexIDs string, digest string,
statsInfos map[string]uint64, copTasks *stmtctx.CopTasksDetails, sql string) string {
statsInfos map[string]uint64, copTasks *stmtctx.CopTasksDetails, memMax int64, sql string) string {
var buf bytes.Buffer
execDetailStr := execDetail.String()
buf.WriteString(SlowLogPrefixStr + SlowLogTxnStartTSStr + SlowLogSpaceMarkStr + strconv.FormatUint(txnTS, 10) + "\n")
Expand Down Expand Up @@ -861,6 +866,9 @@ func (s *SessionVars) SlowLogFormat(txnTS uint64, costTime time.Duration, execDe
fmt.Sprintf("Avg_time: %v P90_time: %v Max_time: %v", copTasks.AvgWaitTime,
copTasks.P90WaitTime, copTasks.MaxWaitTime) + "\n")
}
if memMax > 0 {
buf.WriteString(SlowLogPrefixStr + SlowLogMemMax + SlowLogSpaceMarkStr + strconv.FormatInt(memMax, 10) + "\n")
}
if len(sql) == 0 {
sql = ";"
}
Expand Down
4 changes: 3 additions & 1 deletion sessionctx/variable/session_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -88,6 +88,7 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) {
P90WaitTime: time.Millisecond * 20,
MaxWaitTime: time.Millisecond * 30,
}
var memMax int64 = 2333
resultString := `# Txn_start_ts: 406649736972468225
# User: root@192.168.0.1
# Conn_ID: 1
Expand All @@ -101,9 +102,10 @@ func (*testSessionSuite) TestSlowLogFormat(c *C) {
# Num_cop_tasks: 10
# Cop_process: Avg_time: 1s P90_time: 2s Max_time: 3s
# Cop_wait: Avg_time: 10ms P90_time: 20ms Max_time: 30ms
# Mem_max: 2333
select * from t;`
sql := "select * from t"
digest := parser.DigestHash(sql)
logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[1,2]", digest, statsInfos, copTasks, sql)
logString := seVar.SlowLogFormat(txnTS, costTime, execDetail, "[1,2]", digest, statsInfos, copTasks, memMax, sql)
c.Assert(logString, Equals, resultString)
}
16 changes: 16 additions & 0 deletions util/memory/tracker.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ type Tracker struct {
label string // Label of this "Tracker".
bytesConsumed int64 // Consumed bytes.
bytesLimit int64 // Negative value means no limit.
maxConsumed int64 // max number of bytes consumed during execution.
actionOnExceed ActionOnExceed
parent *Tracker // The parent memory tracker.
children []*Tracker // The children memory trackers.
Expand Down Expand Up @@ -142,6 +143,14 @@ func (t *Tracker) Consume(bytes int64) {
if tracker.bytesLimit > 0 && tracker.bytesConsumed >= tracker.bytesLimit {
rootExceed = tracker
}

if tracker.parent == nil {
// since we only need a total memory usage during execution,
// we only record max consumed bytes in root(statement-level) for performance.
if tracker.bytesConsumed > tracker.maxConsumed {
tracker.maxConsumed = tracker.bytesConsumed
}
}
tracker.Unlock()
}
if rootExceed != nil {
Expand All @@ -156,6 +165,13 @@ func (t *Tracker) BytesConsumed() int64 {
return t.bytesConsumed
}

// MaxConsumed returns max number of bytes consumed during execution.
func (t *Tracker) MaxConsumed() int64 {
t.Lock()
defer t.Unlock()
return t.maxConsumed
}

// String returns the string representation of this Tracker tree.
func (t *Tracker) String() string {
buffer := bytes.NewBufferString("\n")
Expand Down
29 changes: 29 additions & 0 deletions util/memory/tracker_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -14,10 +14,12 @@
package memory

import (
"math/rand"
"os"
"sync"
"testing"

"github.com/cznic/mathutil"
. "github.com/pingcap/check"
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/testleak"
Expand Down Expand Up @@ -190,6 +192,33 @@ func (s *testSuite) TestToString(c *C) {
`)
}

func (s *testSuite) TestMaxConsumed(c *C) {
r := NewTracker("root", -1)
c1 := NewTracker("child 1", -1)
c2 := NewTracker("child 2", -1)
cc1 := NewTracker("child of child 1", -1)

c1.AttachTo(r)
c2.AttachTo(r)
cc1.AttachTo(c1)

ts := []*Tracker{r, c1, c2, cc1}
var consumed, maxConsumed int64
for i := 0; i < 10; i++ {
t := ts[rand.Intn(len(ts))]
b := rand.Int63n(1000) - 500
if consumed+b < 0 {
b = -consumed
}
consumed += b
t.Consume(b)
maxConsumed = mathutil.MaxInt64(maxConsumed, consumed)

c.Assert(r.BytesConsumed(), Equals, consumed)
c.Assert(r.MaxConsumed(), Equals, maxConsumed)
}
}

func BenchmarkConsume(b *testing.B) {
tracker := NewTracker("root", -1)
b.RunParallel(func(pb *testing.PB) {
Expand Down

0 comments on commit dd64171

Please sign in to comment.