Skip to content

Commit

Permalink
executor: fix data race in the slowQueryRuntimeStats
Browse files Browse the repository at this point in the history
Signed-off-by: Weizhen Wang <[email protected]>
  • Loading branch information
hawkingrei committed Dec 5, 2022
1 parent 7012727 commit 44a45cd
Show file tree
Hide file tree
Showing 3 changed files with 7 additions and 16 deletions.
3 changes: 1 addition & 2 deletions executor/executor_pkg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,6 @@ import (
"github.com/pingcap/tidb/util/ranger"
"github.com/pingcap/tidb/util/tableutil"
"github.com/stretchr/testify/require"
"go.uber.org/atomic"
)

// Note: it's a tricky way to export the `inspectionSummaryRules` and `inspectionRules` for unit test but invisible for normal code
Expand Down Expand Up @@ -152,7 +151,7 @@ func TestSlowQueryRuntimeStats(t *testing.T) {
stats := &slowQueryRuntimeStats{
totalFileNum: 2,
readFileNum: 2,
readFile: *atomic.NewDuration(time.Second),
readFile: time.Second,
initialize: time.Millisecond,
readFileSize: 1024 * 1024 * 1024,
parseLog: int64(time.Millisecond * 100),
Expand Down
2 changes: 1 addition & 1 deletion executor/memtable_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -138,7 +138,7 @@ func (e *MemTableReaderExec) Next(ctx context.Context, req *chunk.Chunk) error {
// Close implements the Executor Close interface.
func (e *MemTableReaderExec) Close() error {
if stats := e.retriever.getRuntimeStats(); stats != nil && e.runtimeStats != nil {
e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, stats)
defer e.ctx.GetSessionVars().StmtCtx.RuntimeStatsColl.RegisterStats(e.id, stats)
}
return e.retriever.close()
}
Expand Down
18 changes: 5 additions & 13 deletions executor/slow_query.go
Original file line number Diff line number Diff line change
Expand Up @@ -48,7 +48,6 @@ import (
"github.com/pingcap/tidb/util/logutil"
"github.com/pingcap/tidb/util/memory"
"github.com/pingcap/tidb/util/plancodec"
atomicutil "go.uber.org/atomic"
"go.uber.org/zap"
"golang.org/x/exp/slices"
)
Expand Down Expand Up @@ -471,7 +470,7 @@ func (e *slowQueryRetriever) parseSlowLog(ctx context.Context, sctx sessionctx.C
break
}
if e.stats != nil {
e.stats.readFile.Add(time.Since(startTime))
e.stats.readFile += time.Since(startTime)
}
failpoint.Inject("mockReadSlowLogSlow", func(val failpoint.Value) {
if val.(bool) {
Expand Down Expand Up @@ -1004,7 +1003,7 @@ func (e *slowQueryRetriever) getRuntimeStats() execdetails.RuntimeStats {
type slowQueryRuntimeStats struct {
totalFileNum int
readFileNum int
readFile atomicutil.Duration
readFile time.Duration
initialize time.Duration
readFileSize int64
parseLog int64
Expand All @@ -1014,7 +1013,7 @@ 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",
execdetails.FormatDuration(s.initialize), execdetails.FormatDuration(s.readFile.Load()),
execdetails.FormatDuration(s.initialize), execdetails.FormatDuration(s.readFile),
execdetails.FormatDuration(time.Duration(s.parseLog)), s.concurrent,
s.totalFileNum, s.readFileNum, memory.FormatBytes(s.readFileSize))
}
Expand All @@ -1027,22 +1026,15 @@ func (s *slowQueryRuntimeStats) Merge(rs execdetails.RuntimeStats) {
}
s.totalFileNum += tmp.totalFileNum
s.readFileNum += tmp.readFileNum
s.readFile.Add(tmp.readFile.Load())
s.readFile += tmp.readFile
s.initialize += tmp.initialize
s.readFileSize += tmp.readFileSize
s.parseLog += tmp.parseLog
}

// Clone implements the RuntimeStats interface.
func (s *slowQueryRuntimeStats) Clone() execdetails.RuntimeStats {
var newRs slowQueryRuntimeStats
newRs.totalFileNum = s.totalFileNum
newRs.readFileNum = s.readFileNum
newRs.readFile.Store(s.readFile.Load())
newRs.initialize = s.initialize
newRs.readFileSize = s.readFileSize
newRs.parseLog = s.parseLog
newRs.concurrent = s.concurrent
newRs := *s
return &newRs
}

Expand Down

0 comments on commit 44a45cd

Please sign in to comment.