Skip to content

Commit

Permalink
executor: tiny optimize slow_query performance and add related runtim…
Browse files Browse the repository at this point in the history
…e stats (#20200) (#20420)

Signed-off-by: ti-srebot <[email protected]>
Signed-off-by: crazycs520 <[email protected]>
  • Loading branch information
ti-srebot authored Oct 22, 2020
1 parent 639d459 commit d1454b1
Show file tree
Hide file tree
Showing 6 changed files with 246 additions and 52 deletions.
17 changes: 17 additions & 0 deletions executor/executor_pkg_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package executor
import (
"context"
"crypto/tls"
"time"

. "github.com/pingcap/check"
"github.com/pingcap/failpoint"
Expand Down Expand Up @@ -391,3 +392,19 @@ func (s *testExecSerialSuite) TestSortSpillDisk(c *C) {
err = exec.Close()
c.Assert(err, IsNil)
}

func (s *pkgTestSuite) TestSlowQueryRuntimeStats(c *C) {
stats := &slowQueryRuntimeStats{
totalFileNum: 2,
readFileNum: 2,
readFile: time.Second,
initialize: time.Millisecond,
readFileSize: 1024 * 1024 * 1024,
parseLog: int64(time.Millisecond * 100),
concurrent: 15,
}
c.Assert(stats.String(), Equals, "initialize: 1ms, read_file: 1s, parse_log: {time:100ms, concurrency:15}, total_file: 2, read_file: 2, read_size: 1024 MB")
c.Assert(stats.String(), Equals, stats.Clone().String())
stats.Merge(stats.Clone())
c.Assert(stats.String(), Equals, "initialize: 2ms, read_file: 2s, parse_log: {time:200ms, concurrency:15}, total_file: 4, read_file: 4, read_size: 2 GB")
}
31 changes: 31 additions & 0 deletions executor/executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@ import (
"context"
"flag"
"fmt"
"io/ioutil"
"math"
"net"
"os"
Expand Down Expand Up @@ -5967,6 +5968,36 @@ func (s *testClusterTableSuite) TestSlowQueryWithoutSlowLog(c *C) {
tk.MustQuery("select query from information_schema.slow_query where time > '2020-09-15 12:16:39' and time < now()").Check(testkit.Rows())
}

func (s *testClusterTableSuite) TestSlowQuery2(c *C) {
tk := testkit.NewTestKit(c, s.store)

f, err := ioutil.TempFile("", "tidb-slow-*.log")
c.Assert(err, IsNil)
f.WriteString(`
# Time: 2020-10-13T20:08:13.970563+08:00
select * from t;
# Time: 2020-10-16T20:08:13.970563+08:00
select * from t;
`)
f.Close()

executor.ParseSlowLogBatchSize = 1
originCfg := config.GetGlobalConfig()
newCfg := *originCfg
newCfg.Log.SlowQueryFile = f.Name()
config.StoreGlobalConfig(&newCfg)
defer func() {
executor.ParseSlowLogBatchSize = 64
config.StoreGlobalConfig(originCfg)
os.Remove(newCfg.Log.SlowQueryFile)
}()
err = logutil.InitLogger(newCfg.Log.ToLogConfig())
c.Assert(err, IsNil)

tk.MustQuery("select count(*) from `information_schema`.`slow_query` where time > '2020-10-16 20:08:13' and time < '2020-10-16 21:08:13'").Check(testkit.Rows("1"))
tk.MustQuery("select count(*) from `information_schema`.`slow_query` where time > '2019-10-13 20:08:13' and time < '2020-10-16 21:08:13'").Check(testkit.Rows("2"))
}

func (s *testSplitTable) TestKillTableReader(c *C) {
var retry = "github.com/pingcap/tidb/store/tikv/mockRetrySendReqToRegion"
defer func() {
Expand Down
11 changes: 11 additions & 0 deletions executor/memtable_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,7 @@ import (
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/chunk"
"github.com/pingcap/tidb/util/execdetails"
"github.com/pingcap/tidb/util/pdapi"
"github.com/pingcap/tidb/util/set"
"go.uber.org/zap"
Expand All @@ -53,9 +54,12 @@ type dummyCloser struct{}

func (dummyCloser) close() error { return nil }

func (dummyCloser) getRuntimeStats() execdetails.RuntimeStats { return nil }

type memTableRetriever interface {
retrieve(ctx context.Context, sctx sessionctx.Context) ([][]types.Datum, error)
close() error
getRuntimeStats() execdetails.RuntimeStats
}

// MemTableReaderExec executes memTable information retrieving from the MemTable components
Expand Down Expand Up @@ -127,6 +131,9 @@ 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)
}
return e.retriever.close()
}

Expand Down Expand Up @@ -661,3 +668,7 @@ func (e *clusterLogRetriever) close() error {
}
return nil
}

func (e *clusterLogRetriever) getRuntimeStats() execdetails.RuntimeStats {
return nil
}
Loading

0 comments on commit d1454b1

Please sign in to comment.