Skip to content

Commit

Permalink
*: fix the bug about the insertRuntimeStat.String() (#20190)
Browse files Browse the repository at this point in the history
Signed-off-by: jyz0309 <45495947@qq.com>
  • Loading branch information
jyz0309 authored Sep 27, 2020
1 parent 7e534d9 commit 0017a17
Show file tree
Hide file tree
Showing 4 changed files with 60 additions and 30 deletions.
6 changes: 3 additions & 3 deletions executor/insert.go
Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ func (e *InsertExec) exec(ctx context.Context, rows [][]types.Datum) error {
}
}
if e.stats != nil {
e.stats.checkInsertTime += time.Since(start)
e.stats.CheckInsertTime += time.Since(start)
}
}
e.memTracker.Consume(int64(txn.Size() - txnSize))
Expand Down Expand Up @@ -214,7 +214,7 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D
return err
}
if e.stats != nil {
e.stats.prefetch += time.Since(prefetchStart)
e.stats.Prefetch += time.Since(prefetchStart)
}
for i, r := range toBeCheckedRows {
if r.handleKey != nil {
Expand Down Expand Up @@ -274,7 +274,7 @@ func (e *InsertExec) batchUpdateDupRows(ctx context.Context, newRows [][]types.D
}
}
if e.stats != nil {
e.stats.checkInsertTime += time.Since(start)
e.stats.CheckInsertTime += time.Since(start)
}
return nil
}
Expand Down
65 changes: 39 additions & 26 deletions executor/insert_common.go
Original file line number Diff line number Diff line change
Expand Up @@ -79,7 +79,7 @@ type InsertValues struct {
lazyFillAutoID bool
memTracker *memory.Tracker

stats *insertRuntimeStat
stats *InsertRuntimeStat
}

type defaultVal struct {
Expand Down Expand Up @@ -937,11 +937,11 @@ func (e *InsertValues) collectRuntimeStatsEnabled() bool {
if e.runtimeStats != nil {
if e.stats == nil {
snapshotStats := &tikv.SnapshotRuntimeStats{}
e.stats = &insertRuntimeStat{
e.stats = &InsertRuntimeStat{
BasicRuntimeStats: e.runtimeStats,
SnapshotRuntimeStats: snapshotStats,
prefetch: 0,
checkInsertTime: 0,
Prefetch: 0,
CheckInsertTime: 0,
}
e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, e.stats)
}
Expand Down Expand Up @@ -983,7 +983,7 @@ func (e *InsertValues) batchCheckAndInsert(ctx context.Context, rows [][]types.D
return err
}
if e.stats != nil {
e.stats.prefetch += time.Since(prefetchStart)
e.stats.Prefetch += time.Since(prefetchStart)
}

// append warnings and get no duplicated error rows
Expand Down Expand Up @@ -1024,7 +1024,7 @@ func (e *InsertValues) batchCheckAndInsert(ctx context.Context, rows [][]types.D
}
}
if e.stats != nil {
e.stats.checkInsertTime += time.Since(start)
e.stats.CheckInsertTime += time.Since(start)
}
return nil
}
Expand Down Expand Up @@ -1053,51 +1053,56 @@ func (e *InsertValues) addRecordWithAutoIDHint(ctx context.Context, row []types.
return nil
}

type insertRuntimeStat struct {
// InsertRuntimeStat record the stat about insert and check
type InsertRuntimeStat struct {
*execdetails.BasicRuntimeStats
*tikv.SnapshotRuntimeStats
checkInsertTime time.Duration
prefetch time.Duration
CheckInsertTime time.Duration
Prefetch time.Duration
}

func (e *insertRuntimeStat) String() string {
if e.checkInsertTime == 0 {
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())
if e.Prefetch > 0 && e.SnapshotRuntimeStats != nil {
return fmt.Sprintf("prefetch: %v, rpc:{%v}", 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))
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("prepare:%v, ", 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))
if e.SnapshotRuntimeStats != nil {
buf.WriteString(fmt.Sprintf(", rpc:{%s}", e.SnapshotRuntimeStats.String()))
}
buf.WriteString("}")
} else {
buf.WriteString(fmt.Sprintf("insert:%v", e.checkInsertTime))
buf.WriteString(fmt.Sprintf("insert:%v", e.CheckInsertTime))
}
return buf.String()
}

// Clone implements the RuntimeStats interface.
func (e *insertRuntimeStat) Clone() execdetails.RuntimeStats {
newRs := &insertRuntimeStat{
checkInsertTime: e.checkInsertTime,
prefetch: e.prefetch,
func (e *InsertRuntimeStat) Clone() execdetails.RuntimeStats {
newRs := &InsertRuntimeStat{
CheckInsertTime: e.CheckInsertTime,
Prefetch: e.Prefetch,
}
if e.SnapshotRuntimeStats != nil {
snapshotStats := e.SnapshotRuntimeStats.Clone()
newRs.SnapshotRuntimeStats = snapshotStats.(*tikv.SnapshotRuntimeStats)
}
if e.BasicRuntimeStats != nil {
basicStats := e.BasicRuntimeStats.Clone()
newRs.BasicRuntimeStats = basicStats.(*execdetails.BasicRuntimeStats)
}
return newRs
}

// Merge implements the RuntimeStats interface.
func (e *insertRuntimeStat) Merge(other execdetails.RuntimeStats) {
tmp, ok := other.(*insertRuntimeStat)
func (e *InsertRuntimeStat) Merge(other execdetails.RuntimeStats) {
tmp, ok := other.(*InsertRuntimeStat)
if !ok {
return
}
Expand All @@ -1109,11 +1114,19 @@ func (e *insertRuntimeStat) Merge(other execdetails.RuntimeStats) {
e.SnapshotRuntimeStats.Merge(tmp.SnapshotRuntimeStats)
}
}
e.prefetch += tmp.prefetch
e.checkInsertTime += tmp.checkInsertTime
if tmp.BasicRuntimeStats != nil {
if e.BasicRuntimeStats == nil {
basicStats := tmp.BasicRuntimeStats.Clone()
e.BasicRuntimeStats = basicStats.(*execdetails.BasicRuntimeStats)
} else {
e.BasicRuntimeStats.Merge(tmp.BasicRuntimeStats)
}
}
e.Prefetch += tmp.Prefetch
e.CheckInsertTime += tmp.CheckInsertTime
}

// Tp implements the RuntimeStats interface.
func (e *insertRuntimeStat) Tp() int {
func (e *InsertRuntimeStat) Tp() int {
return execdetails.TpInsertRuntimeStat
}
17 changes: 17 additions & 0 deletions executor/insert_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,18 @@ import (
"strconv"
"strings"
"sync"
"time"

. "github.com/pingcap/check"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/errno"
"github.com/pingcap/tidb/executor"
"github.com/pingcap/tidb/meta/autoid"
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/collate"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/testkit"
"github.com/pingcap/tidb/util/testutil"
)
Expand Down Expand Up @@ -1319,6 +1322,20 @@ func (s *testSuite10) TestClusterPrimaryKeyForIndexScan(c *C) {
c.Assert(cnt, Equals, 15)
}

func (s *testSuite10) TestInsertRuntimeStat(c *C) {
stats := &executor.InsertRuntimeStat{
BasicRuntimeStats: &execdetails.BasicRuntimeStats{},
SnapshotRuntimeStats: nil,
CheckInsertTime: 2 * time.Second,
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, 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}")
}

func (s *testSerialSuite) TestDuplicateEntryMessage(c *C) {
collate.SetNewCollationEnabledForTest(true)
defer collate.SetNewCollationEnabledForTest(false)
Expand Down
2 changes: 1 addition & 1 deletion executor/replace.go
Original file line number Diff line number Diff line change
Expand Up @@ -208,7 +208,7 @@ func (e *ReplaceExec) exec(ctx context.Context, newRows [][]types.Datum) error {
return err
}
if e.stats != nil {
e.stats.prefetch = time.Since(prefetchStart)
e.stats.Prefetch = time.Since(prefetchStart)
}
e.ctx.GetSessionVars().StmtCtx.AddRecordRows(uint64(len(newRows)))
for _, r := range toBeCheckedRows {
Expand Down

0 comments on commit 0017a17

Please sign in to comment.