Skip to content

Commit

Permalink
planner, infoschema : add columns about plan cache in 'performance_sc…
Browse files Browse the repository at this point in the history
…hema.events_statements_summary_by_digest' (#16476) (#17493)
  • Loading branch information
sre-bot authored Jun 1, 2020
1 parent 7683951 commit d150d38
Show file tree
Hide file tree
Showing 7 changed files with 77 additions and 10 deletions.
1 change: 1 addition & 0 deletions executor/adapter.go
Original file line number Diff line number Diff line change
Expand Up @@ -977,5 +977,6 @@ func (a *ExecStmt) SummaryStmt(succ bool) {
StartTime: sessVars.StartTime,
IsInternal: sessVars.InRestrictedSQL,
Succeed: succ,
PlanInCache: sessVars.FoundInPlanCache,
})
}
2 changes: 2 additions & 0 deletions infoschema/perfschema/const.go
Original file line number Diff line number Diff line change
Expand Up @@ -413,6 +413,8 @@ const tableEventsStatementsSummaryByDigest = "CREATE TABLE if not exists perform
"SUM_NO_GOOD_INDEX_USED bigint unsigned NOT NULL," +
"FIRST_SEEN timestamp(6) NOT NULL DEFAULT '0000-00-00 00:00:00.000000'," +
"LAST_SEEN timestamp(6) NOT NULL DEFAULT '0000-00-00 00:00:00.000000'," +
"PLAN_IN_CACHE bool NOT NULL," +
"PLAN_CACHE_HITS bigint unsigned NOT NULL," +
"QUANTILE_95 bigint unsigned NOT NULL," +
"QUANTILE_99 bigint unsigned NOT NULL," +
"QUANTILE_999 bigint unsigned NOT NULL," +
Expand Down
2 changes: 2 additions & 0 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -1194,6 +1194,8 @@ var tableStatementsSummaryCols = []columnInfo{
{name: "AVG_AFFECTED_ROWS", tp: mysql.TypeDouble, size: 22, flag: mysql.NotNullFlag | mysql.UnsignedFlag, comment: "Average number of rows affected"},
{name: "FIRST_SEEN", tp: mysql.TypeTimestamp, size: 26, flag: mysql.NotNullFlag, comment: "The time these statements are seen for the first time"},
{name: "LAST_SEEN", tp: mysql.TypeTimestamp, size: 26, flag: mysql.NotNullFlag, comment: "The time these statements are seen for the last time"},
{name: "PLAN_IN_CACHE", tp: mysql.TypeTiny, size: 1, flag: mysql.NotNullFlag, comment: "Whether the last statement hit plan cache"},
{name: "PLAN_CACHE_HITS", tp: mysql.TypeLonglong, size: 20, flag: mysql.NotNullFlag, comment: "The number of times these statements hit plan cache"},
{name: "QUERY_SAMPLE_TEXT", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "Sampled original statement"},
{name: "PREV_SAMPLE_TEXT", tp: mysql.TypeBlob, size: types.UnspecifiedLength, comment: "The previous statement before commit"},
{name: "PLAN_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of its execution plan"},
Expand Down
41 changes: 41 additions & 0 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,6 +16,7 @@ package infoschema_test
import (
"crypto/tls"
"fmt"
"math"
"net"
"net/http/httptest"
"os"
Expand All @@ -36,11 +37,13 @@ import (
"github.com/pingcap/tidb/infoschema"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/meta/autoid"
plannercore "github.com/pingcap/tidb/planner/core"
"github.com/pingcap/tidb/server"
"github.com/pingcap/tidb/session"
"github.com/pingcap/tidb/store/helper"
"github.com/pingcap/tidb/store/mockstore"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/kvcache"
"github.com/pingcap/tidb/util/pdapi"
"github.com/pingcap/tidb/util/set"
"github.com/pingcap/tidb/util/testkit"
Expand Down Expand Up @@ -84,6 +87,18 @@ func (s *testTableSuiteBase) newTestKitWithRoot(c *C) *testkit.TestKit {
return tk
}

func (s *testTableSuiteBase) newTestKitWithPlanCache(c *C) *testkit.TestKit {
tk := testkit.NewTestKit(c, s.store)
var err error
tk.Se, err = session.CreateSession4TestWithOpt(s.store, &session.Opt{
PreparedPlanCache: kvcache.NewSimpleLRUCache(100, 0.1, math.MaxUint64),
})
c.Assert(err, IsNil)
tk.GetConnectionID()
c.Assert(tk.Se.Auth(&auth.UserIdentity{Username: "root", Hostname: "%"}, nil, nil), IsTrue)
return tk
}

type testClusterTableSuite struct {
*testTableSuiteBase
rpcserver *grpc.Server
Expand Down Expand Up @@ -1231,3 +1246,29 @@ func (s *testTableSuite) TestStmtSummaryPreparedStatements(c *C) {
from information_schema.statements_summary
where digest_text like "select ?"`).Check(testkit.Rows("1"))
}

func (s *testTableSuite) TestPerformanceSchemaforPlanCache(c *C) {
orgEnable := plannercore.PreparedPlanCacheEnabled()
defer func() {
plannercore.SetPreparedPlanCache(orgEnable)
}()
plannercore.SetPreparedPlanCache(true)

tk := s.newTestKitWithPlanCache(c)

// Clear summaries.
tk.MustExec("set global tidb_enable_stmt_summary = 0")
tk.MustExec("set global tidb_enable_stmt_summary = 1")
tk.MustExec("use test")
tk.MustExec("drop table if exists t")
tk.MustExec("create table t(a int)")
tk.MustExec("prepare stmt from 'select * from t'")
tk.MustExec("execute stmt")
tk.MustQuery("select plan_cache_hits, plan_in_cache from information_schema.statements_summary where digest_text='select * from t'").Check(
testkit.Rows("0 0"))
tk.MustExec("execute stmt")
tk.MustExec("execute stmt")
tk.MustExec("execute stmt")
tk.MustQuery("select plan_cache_hits, plan_in_cache from information_schema.statements_summary where digest_text='select * from t'").Check(
testkit.Rows("3 1"))
}
28 changes: 21 additions & 7 deletions util/stmtsummary/statement_summary.go
Original file line number Diff line number Diff line change
Expand Up @@ -168,6 +168,9 @@ type stmtSummaryByDigestElement struct {
firstSeen time.Time
// The last time this type of SQL executes.
lastSeen time.Time
// plan cache
planInCache bool
planCacheHits int64
}

// StmtExecInfo records execution information of each statement.
Expand All @@ -192,6 +195,7 @@ type StmtExecInfo struct {
StartTime time.Time
IsInternal bool
Succeed bool
PlanInCache bool
}

// newStmtSummaryByDigestMap creates an empty stmtSummaryByDigestMap.
Expand Down Expand Up @@ -568,13 +572,15 @@ func newStmtSummaryByDigestElement(sei *StmtExecInfo, beginTime int64, intervalS
// PrevSQL is already truncated to cfg.Log.QueryLogMaxLen.
prevSQL: sei.PrevSQL,
// samplePlan needs to be decoded so it can't be truncated.
samplePlan: sei.PlanGenerator(),
indexNames: sei.StmtCtx.IndexNames,
minLatency: sei.TotalLatency,
firstSeen: sei.StartTime,
lastSeen: sei.StartTime,
backoffTypes: make(map[fmt.Stringer]int),
authUsers: make(map[string]struct{}),
samplePlan: sei.PlanGenerator(),
indexNames: sei.StmtCtx.IndexNames,
minLatency: sei.TotalLatency,
firstSeen: sei.StartTime,
lastSeen: sei.StartTime,
backoffTypes: make(map[fmt.Stringer]int),
authUsers: make(map[string]struct{}),
planInCache: false,
planCacheHits: 0,
}
ssElement.add(sei, intervalSeconds)
return ssElement
Expand Down Expand Up @@ -721,6 +727,12 @@ func (ssElement *stmtSummaryByDigestElement) add(sei *StmtExecInfo, intervalSeco
commitDetails.Mu.Unlock()
}

//plan cache
if sei.PlanInCache {
ssElement.planInCache = true
ssElement.planCacheHits += 1
}

// other
ssElement.sumAffectedRows += sei.StmtCtx.AffectedRows()
ssElement.sumMem += sei.MemMax
Expand Down Expand Up @@ -815,6 +827,8 @@ func (ssElement *stmtSummaryByDigestElement) toDatum(ssbd *stmtSummaryByDigest)
avgFloat(int64(ssElement.sumAffectedRows), ssElement.execCount),
types.NewTime(types.FromGoTime(ssElement.firstSeen), mysql.TypeTimestamp, 0),
types.NewTime(types.FromGoTime(ssElement.lastSeen), mysql.TypeTimestamp, 0),
ssElement.planInCache,
ssElement.planCacheHits,
ssElement.sampleSQL,
ssElement.prevSQL,
ssbd.planDigest,
Expand Down
2 changes: 1 addition & 1 deletion util/stmtsummary/statement_summary_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -606,7 +606,7 @@ func (s *testStmtSummarySuite) TestToDatum(c *C) {
stmtExecInfo1.ExecDetail.CommitDetail.PrewriteRegionNum, stmtExecInfo1.ExecDetail.CommitDetail.PrewriteRegionNum,
stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, stmtExecInfo1.ExecDetail.CommitDetail.TxnRetry, 1,
"txnLock:1", stmtExecInfo1.MemMax, stmtExecInfo1.MemMax, stmtExecInfo1.StmtCtx.AffectedRows(),
t, t, stmtExecInfo1.OriginalSQL, stmtExecInfo1.PrevSQL, "plan_digest", ""}
t, t, 0, 0, stmtExecInfo1.OriginalSQL, stmtExecInfo1.PrevSQL, "plan_digest", ""}
match(c, datums[0], expectedDatum...)

datums = s.ssMap.ToHistoryDatum(nil, true)
Expand Down
11 changes: 9 additions & 2 deletions util/testkit/testkit.go
Original file line number Diff line number Diff line change
Expand Up @@ -129,14 +129,21 @@ func NewTestKitWithInit(c *check.C, store kv.Storage) *TestKit {

var connectionID uint64

// GetConnectionID get the connection ID for tk.Se
func (tk *TestKit) GetConnectionID() {
if tk.Se != nil {
id := atomic.AddUint64(&connectionID, 1)
tk.Se.SetConnectionID(id)
}
}

// Exec executes a sql statement.
func (tk *TestKit) Exec(sql string, args ...interface{}) (sqlexec.RecordSet, error) {
var err error
if tk.Se == nil {
tk.Se, err = session.CreateSession4Test(tk.store)
tk.c.Assert(err, check.IsNil)
id := atomic.AddUint64(&connectionID, 1)
tk.Se.SetConnectionID(id)
tk.GetConnectionID()
}
ctx := context.Background()
if len(args) == 0 {
Expand Down

0 comments on commit d150d38

Please sign in to comment.