Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

executor: tiny refactor the runtimeStats collection code #10917

Merged
merged 3 commits into from
Jun 24, 2019
Merged
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Next Next commit
executor: tiny refactor the runtimeStats collection code
remove redundant code and do it in one place
  • Loading branch information
tiancaiamao committed Jun 24, 2019
commit dae1a41fda94697dfb31996fc577eb9822ce4085
9 changes: 0 additions & 9 deletions executor/aggregate.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ package executor
import (
"context"
"sync"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
Expand Down Expand Up @@ -514,10 +513,6 @@ func (w *HashAggFinalWorker) run(ctx sessionctx.Context, waitGroup *sync.WaitGro

// Next implements the Executor Next interface.
func (e *HashAggExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.Reset()
if e.isUnparallelExec {
return e.unparallelExec(ctx, req.Chunk)
Expand Down Expand Up @@ -790,10 +785,6 @@ func (e *StreamAggExec) Close() error {

// Next implements the Executor Next interface.
func (e *StreamAggExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.Reset()
for !e.executed && !req.IsFull() {
err := e.consumeOneGroup(ctx, req.Chunk)
Expand Down
9 changes: 0 additions & 9 deletions executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -24,7 +24,6 @@ import (
"time"
"unsafe"

"github.com/opentracing/opentracing-go"
"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
"github.com/pingcap/parser/mysql"
Expand Down Expand Up @@ -269,14 +268,6 @@ func (e *IndexReaderExecutor) Close() error {

// Next implements the Executor Next interface.
func (e *IndexReaderExecutor) Next(ctx context.Context, req *chunk.RecordBatch) error {
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan("tableReader.Next", opentracing.ChildOf(span.Context()))
defer span1.Finish()
}
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
err := e.result.Next(ctx, req.Chunk)
if err != nil {
e.feedback.Invalidate()
Expand Down
35 changes: 6 additions & 29 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,14 +182,19 @@ type Executor interface {

// Next is a wrapper function on e.Next(), it handles some common codes.
func Next(ctx context.Context, e Executor, req *chunk.RecordBatch) error {
sessVars := e.base().ctx.GetSessionVars()
base := e.base()
sessVars := base.ctx.GetSessionVars()
if atomic.CompareAndSwapUint32(&sessVars.Killed, 1, 0) {
return ErrQueryInterrupted
}
if span := opentracing.SpanFromContext(ctx); span != nil && span.Tracer() != nil {
span1 := span.Tracer().StartSpan(fmt.Sprintf("%T.Next", e), opentracing.ChildOf(span.Context()))
defer span1.Finish()
}
if base.runtimeStats != nil {
start := time.Now()
defer func() { base.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
return e.Next(ctx, req)
}

Expand All @@ -204,10 +209,6 @@ type CancelDDLJobsExec struct {

// Next implements the Executor Next interface.
func (e *CancelDDLJobsExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.GrowAndReset(e.maxChunkSize)
if e.cursor >= len(e.jobIDs) {
return nil
Expand Down Expand Up @@ -721,10 +722,6 @@ type LimitExec struct {

// Next implements the Executor Next interface.
func (e *LimitExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.Reset()
if e.cursor >= e.end {
return nil
Expand Down Expand Up @@ -861,10 +858,6 @@ func (e *TableDualExec) Open(ctx context.Context) error {

// Next implements the Executor Next interface.
func (e *TableDualExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.Reset()
if e.numReturned >= e.numDualRows {
return nil
Expand Down Expand Up @@ -916,10 +909,6 @@ func (e *SelectionExec) Close() error {

// Next implements the Executor Next interface.
func (e *SelectionExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.GrowAndReset(e.maxChunkSize)

if !e.batched {
Expand Down Expand Up @@ -995,10 +984,6 @@ type TableScanExec struct {

// Next implements the Executor Next interface.
func (e *TableScanExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.GrowAndReset(e.maxChunkSize)
if e.isVirtualTable {
return e.nextChunk4InfoSchema(ctx, req.Chunk)
Expand Down Expand Up @@ -1099,10 +1084,6 @@ func (e *MaxOneRowExec) Open(ctx context.Context) error {

// Next implements the Executor Next interface.
func (e *MaxOneRowExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.Reset()
if e.evaluated {
return nil
Expand Down Expand Up @@ -1248,10 +1229,6 @@ func (e *UnionExec) resultPuller(ctx context.Context, childID int) {

// Next implements the Executor Next interface.
func (e *UnionExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.GrowAndReset(e.maxChunkSize)
if !e.initialized {
e.initialize(ctx)
Expand Down
5 changes: 0 additions & 5 deletions executor/index_lookup_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,6 @@ import (
"sort"
"sync"
"sync/atomic"
"time"
"unsafe"

"github.com/pingcap/errors"
Expand Down Expand Up @@ -228,10 +227,6 @@ func (e *IndexLookUpJoin) newInnerWorker(taskCh chan *lookUpJoinTask) *innerWork

// Next implements the Executor interface.
func (e *IndexLookUpJoin) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
if e.isOuterJoin {
atomic.StoreInt64(&e.requiredRows, int64(req.RequiredRows()))
}
Expand Down
9 changes: 0 additions & 9 deletions executor/join.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"fmt"
"sync"
"sync/atomic"
"time"
"unsafe"

"github.com/pingcap/errors"
Expand Down Expand Up @@ -498,10 +497,6 @@ func (e *HashJoinExec) join2Chunk(workerID uint, outerChk *chunk.Chunk, joinResu
// step 1. fetch data from inner child and build a hash table;
// step 2. fetch data from outer child in a background goroutine and probe the hash table in multiple join workers.
func (e *HashJoinExec) Next(ctx context.Context, req *chunk.RecordBatch) (err error) {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
if !e.prepared {
e.innerFinished = make(chan error, 1)
go util.WithRecovery(func() { e.fetchInnerAndBuildHashTable(ctx) }, e.handleFetchInnerAndBuildHashTablePanic)
Expand Down Expand Up @@ -705,10 +700,6 @@ func (e *NestedLoopApplyExec) fetchAllInners(ctx context.Context) error {

// Next implements the Executor interface.
func (e *NestedLoopApplyExec) Next(ctx context.Context, req *chunk.RecordBatch) (err error) {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.Reset()
for {
if e.innerIter == nil || e.innerIter.Current() == e.innerIter.End() {
Expand Down
5 changes: 0 additions & 5 deletions executor/merge_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ package executor
import (
"context"
"fmt"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/expression"
Expand Down Expand Up @@ -268,10 +267,6 @@ func (e *MergeJoinExec) prepare(ctx context.Context, requiredRows int) error {

// Next implements the Executor Next interface.
func (e *MergeJoinExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.Reset()
if !e.prepared {
if err := e.prepare(ctx, req.RequiredRows()); err != nil {
Expand Down
5 changes: 0 additions & 5 deletions executor/projection.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"context"
"fmt"
"sync/atomic"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/tidb/expression"
Expand Down Expand Up @@ -154,10 +153,6 @@ func (e *ProjectionExec) Open(ctx context.Context) error {
// +------------------------------+ +----------------------+
//
func (e *ProjectionExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.GrowAndReset(e.maxChunkSize)
if e.isUnparallelExec() {
return e.unParallelExecute(ctx, req.Chunk)
Expand Down
5 changes: 0 additions & 5 deletions executor/radix_hash_join.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,7 +17,6 @@ import (
"context"
"math"
"sync"
"time"
"unsafe"

"github.com/pingcap/errors"
Expand Down Expand Up @@ -86,10 +85,6 @@ var partPtr4NullKey = partRowPtr{math.MaxUint32, math.MaxUint32}
// step 4. probe the corresponded sub-hash-table for every sub-outer-relation in
// multiple join workers
func (e *RadixHashJoinExec) Next(ctx context.Context, req *chunk.RecordBatch) (err error) {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Now().Sub(start), req.NumRows()) }()
}
if !e.prepared {
e.innerFinished = make(chan error, 1)
go util.WithRecovery(func() { e.partitionInnerAndBuildHashTables(ctx) }, e.handleFetchInnerAndBuildHashTablePanic)
Expand Down
9 changes: 0 additions & 9 deletions executor/sort.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@ import (
"context"
"fmt"
"sort"
"time"

"github.com/pingcap/tidb/expression"
plannercore "github.com/pingcap/tidb/planner/core"
Expand Down Expand Up @@ -75,10 +74,6 @@ func (e *SortExec) Open(ctx context.Context) error {

// Next implements the Executor Next interface.
func (e *SortExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.Reset()
if !e.fetched {
err := e.fetchRowChunks(ctx)
Expand Down Expand Up @@ -235,10 +230,6 @@ func (e *TopNExec) Open(ctx context.Context) error {

// Next implements the Executor Next interface.
func (e *TopNExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.Reset()
if !e.fetched {
e.totalLimit = e.limit.Offset + e.limit.Count
Expand Down
5 changes: 0 additions & 5 deletions executor/table_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ package executor
import (
"context"
"fmt"
"time"

"github.com/pingcap/parser/model"
"github.com/pingcap/tidb/distsql"
Expand Down Expand Up @@ -131,10 +130,6 @@ func (e *TableReaderExecutor) Open(ctx context.Context) error {
// Next fills data into the chunk passed by its caller.
// The task was actually done by tableReaderHandler.
func (e *TableReaderExecutor) Next(ctx context.Context, req *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
if err := e.resultHandler.nextChunk(ctx, req.Chunk); err != nil {
e.feedback.Invalidate()
return err
Expand Down
5 changes: 0 additions & 5 deletions executor/union_scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ package executor
import (
"context"
"sort"
"time"

"github.com/pingcap/errors"
"github.com/pingcap/parser/model"
Expand Down Expand Up @@ -122,10 +121,6 @@ func (us *UnionScanExec) Open(ctx context.Context) error {

// Next implements the Executor Next interface.
func (us *UnionScanExec) Next(ctx context.Context, req *chunk.RecordBatch) error {
if us.runtimeStats != nil {
start := time.Now()
defer func() { us.runtimeStats.Record(time.Since(start), req.NumRows()) }()
}
req.GrowAndReset(us.maxChunkSize)
mutableRow := chunk.MutRowFromTypes(retTypes(us))
for i, batchSize := 0, req.Capacity(); i < batchSize; i++ {
Expand Down
5 changes: 0 additions & 5 deletions executor/window.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,6 @@ package executor

import (
"context"
"time"

"github.com/cznic/mathutil"
"github.com/pingcap/errors"
Expand Down Expand Up @@ -52,10 +51,6 @@ func (e *WindowExec) Close() error {

// Next implements the Executor Next interface.
func (e *WindowExec) Next(ctx context.Context, chk *chunk.RecordBatch) error {
if e.runtimeStats != nil {
start := time.Now()
defer func() { e.runtimeStats.Record(time.Now().Sub(start), chk.NumRows()) }()
}
chk.Reset()
if e.meetNewGroup && e.remainingRowsInGroup > 0 {
err := e.appendResult2Chunk(chk.Chunk)
Expand Down