Skip to content

Commit

Permalink
inforschema, executor, util/kvcache, util/statement_summary : Add STA…
Browse files Browse the repository at this point in the history
…TEMENTS_SUMMARY_EVICTED into information_schema (#24513)
  • Loading branch information
ClSlaid authored May 28, 2021
1 parent 138692f commit 0367c54
Show file tree
Hide file tree
Showing 9 changed files with 615 additions and 15 deletions.
1 change: 1 addition & 0 deletions executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -1527,6 +1527,7 @@ func (b *executorBuilder) buildMemTable(v *plannercore.PhysicalMemTable) Executo
strings.ToLower(infoschema.TableTiKVStoreStatus),
strings.ToLower(infoschema.TableStatementsSummary),
strings.ToLower(infoschema.TableStatementsSummaryHistory),
strings.ToLower(infoschema.TableStatementsSummaryEvicted),
strings.ToLower(infoschema.ClusterTableStatementsSummary),
strings.ToLower(infoschema.ClusterTableStatementsSummaryHistory),
strings.ToLower(infoschema.TablePlacementPolicy),
Expand Down
6 changes: 6 additions & 0 deletions executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -148,6 +148,8 @@ func (e *memtableRetriever) retrieve(ctx context.Context, sctx sessionctx.Contex
infoschema.ClusterTableStatementsSummary,
infoschema.ClusterTableStatementsSummaryHistory:
err = e.setDataForStatementsSummary(sctx, e.table.Name.O)
case infoschema.TableStatementsSummaryEvicted:
e.setDataForStatementsSummaryEvicted(sctx)
case infoschema.TablePlacementPolicy:
err = e.setDataForPlacementPolicy(sctx)
case infoschema.TableClientErrorsSummaryGlobal,
Expand Down Expand Up @@ -2099,6 +2101,10 @@ func (e *memtableRetriever) setDataForClusterDeadlock(ctx sessionctx.Context) er
return nil
}

func (e *memtableRetriever) setDataForStatementsSummaryEvicted(ctx sessionctx.Context) {
e.rows = stmtsummary.StmtSummaryByDigestMap.ToEvictedCountDatum()
}

type hugeMemTableRetriever struct {
dummyCloser
table *model.TableInfo
Expand Down
10 changes: 10 additions & 0 deletions infoschema/tables.go
Original file line number Diff line number Diff line change
Expand Up @@ -149,6 +149,8 @@ const (
TableStatementsSummary = "STATEMENTS_SUMMARY"
// TableStatementsSummaryHistory is the string constant of statements summary history table.
TableStatementsSummaryHistory = "STATEMENTS_SUMMARY_HISTORY"
// TableStatementsSummaryEvicted is the string constant of statements summary evicted table.
TableStatementsSummaryEvicted = "STATEMENTS_SUMMARY_EVICTED"
// TableStorageStats is a table that contains all tables disk usage
TableStorageStats = "TABLE_STORAGE_STATS"
// TableTiFlashTables is the string constant of tiflash tables table.
Expand Down Expand Up @@ -246,6 +248,7 @@ var tableIDMap = map[string]int64{
TableDeadlocks: autoid.InformationSchemaDBID + 72,
ClusterTableDeadlocks: autoid.InformationSchemaDBID + 73,
TableDataLockWaits: autoid.InformationSchemaDBID + 74,
TableStatementsSummaryEvicted: autoid.InformationSchemaDBID + 75,
}

type columnInfo struct {
Expand Down Expand Up @@ -1380,6 +1383,12 @@ var tableDataLockWaitsCols = []columnInfo{
{name: "SQL_DIGEST", tp: mysql.TypeVarchar, size: 64, comment: "Digest of the SQL that's trying to acquire the lock"},
}

var tableStatementsSummaryEvictedCols = []columnInfo{
{name: "BEGIN_TIME", tp: mysql.TypeTimestamp, size: 26},
{name: "END_TIME", tp: mysql.TypeTimestamp, size: 26},
{name: "EVICTED_COUNT", tp: mysql.TypeLonglong, size: 64, flag: mysql.NotNullFlag},
}

// GetShardingInfo returns a nil or description string for the sharding information of given TableInfo.
// The returned description string may be:
// - "NOT_SHARDED": for tables that SHARD_ROW_ID_BITS is not specified.
Expand Down Expand Up @@ -1742,6 +1751,7 @@ var tableNameToColumns = map[string][]columnInfo{
TableSequences: tableSequencesCols,
TableStatementsSummary: tableStatementsSummaryCols,
TableStatementsSummaryHistory: tableStatementsSummaryCols,
TableStatementsSummaryEvicted: tableStatementsSummaryEvictedCols,
TableStorageStats: tableStorageStatsCols,
TableTiFlashTables: tableTableTiFlashTablesCols,
TableTiFlashSegments: tableTableTiFlashSegmentsCols,
Expand Down
27 changes: 27 additions & 0 deletions infoschema/tables_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1373,6 +1373,33 @@ func (s *testTableSuite) TestStmtSummarySensitiveQuery(c *C) {
))
}

func (s *testTableSuite) TestSimpleStmtSummaryEvictedCount(c *C) {
now := time.Now().Unix()
interval := int64(1800)
beginTimeForCurInterval := now - now%interval
tk := s.newTestKitWithPlanCache(c)
tk.MustExec(fmt.Sprintf("set global tidb_stmt_summary_refresh_interval = %v", interval))
tk.MustExec("set global tidb_enable_stmt_summary = 0")
tk.MustExec("set global tidb_enable_stmt_summary = 1")
// first sql
tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 1")
// second sql
tk.MustQuery("show databases;")
// query `evicted table` is also a SQL, passing it leads to the eviction of the previous SQLs.
tk.MustQuery("select * from `information_schema`.`STATEMENTS_SUMMARY_EVICTED`;").
Check(testkit.Rows(
fmt.Sprintf("%s %s %v",
time.Unix(beginTimeForCurInterval, 0).Format("2006-01-02 15:04:05"),
time.Unix(beginTimeForCurInterval+interval, 0).Format("2006-01-02 15:04:05"),
int64(2)),
))
// TODO: Add more tests.

// clean up side effects
tk.MustExec("set global tidb_stmt_summary_max_stmt_count = 100")
tk.MustExec("set global tidb_stmt_summary_refresh_interval = 1800")
}

func (s *testTableSuite) TestPerformanceSchemaforPlanCache(c *C) {
orgEnable := plannercore.PreparedPlanCacheEnabled()
defer func() {
Expand Down
3 changes: 3 additions & 0 deletions util/kvcache/simple_lru.go
Original file line number Diff line number Diff line change
Expand Up @@ -118,6 +118,9 @@ func (l *SimpleLRUCache) Put(key Key, value Value) {
if l.size > l.capacity {
lru := l.cache.Back()
l.cache.Remove(lru)
if l.onEvict != nil {
l.onEvict(lru.Value.(*cacheEntry).key, lru.Value.(*cacheEntry).value)
}
delete(l.elements, string(lru.Value.(*cacheEntry).key.Hash()))
l.size--
}
Expand Down
38 changes: 25 additions & 13 deletions util/kvcache/simple_lru_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,35 +58,47 @@ func (s *testLRUCacheSuite) TestPut(c *C) {
maxMem, err := memory.MemTotal()
c.Assert(err, IsNil)

lru := NewSimpleLRUCache(3, 0, maxMem)
c.Assert(lru.capacity, Equals, uint(3))
lruMaxMem := NewSimpleLRUCache(3, 0, maxMem)
lruZeroQuota := NewSimpleLRUCache(3, 0, 0)
c.Assert(lruMaxMem.capacity, Equals, uint(3))
c.Assert(lruZeroQuota.capacity, Equals, uint(3))

keys := make([]*mockCacheKey, 5)
vals := make([]int64, 5)
droppedKv := make(map[Key]Value)
maxMemDroppedKv := make(map[Key]Value)
zeroQuotaDroppedKv := make(map[Key]Value)

lru.SetOnEvict(func(key Key, value Value) {
droppedKv[key] = value
// test onEvict function
lruMaxMem.SetOnEvict(func(key Key, value Value) {
maxMemDroppedKv[key] = value
})
// test onEvict function on 0 value of quota
lruZeroQuota.SetOnEvict(func(key Key, value Value) {
zeroQuotaDroppedKv[key] = value
})
for i := 0; i < 5; i++ {
keys[i] = newMockHashKey(int64(i))
vals[i] = int64(i)
lru.Put(keys[i], vals[i])
lruMaxMem.Put(keys[i], vals[i])
lruZeroQuota.Put(keys[i], vals[i])
}
c.Assert(lru.size, Equals, lru.capacity)
c.Assert(lru.size, Equals, uint(3))
c.Assert(lruMaxMem.size, Equals, lruMaxMem.capacity)
c.Assert(lruZeroQuota.size, Equals, lruZeroQuota.capacity)
c.Assert(lruMaxMem.size, Equals, uint(3))
c.Assert(lruZeroQuota.size, Equals, lruMaxMem.size)

// test for non-existent elements
c.Assert(len(droppedKv), Equals, 2)
c.Assert(len(maxMemDroppedKv), Equals, 2)
for i := 0; i < 2; i++ {
element, exists := lru.elements[string(keys[i].Hash())]
element, exists := lruMaxMem.elements[string(keys[i].Hash())]
c.Assert(exists, IsFalse)
c.Assert(element, IsNil)
c.Assert(droppedKv[keys[i]], Equals, vals[i])
c.Assert(maxMemDroppedKv[keys[i]], Equals, vals[i])
c.Assert(zeroQuotaDroppedKv[keys[i]], Equals, vals[i])
}

// test for existent elements
root := lru.cache.Front()
root := lruMaxMem.cache.Front()
c.Assert(root, NotNil)
for i := 4; i >= 2; i-- {
entry, ok := root.Value.(*cacheEntry)
Expand All @@ -98,7 +110,7 @@ func (s *testLRUCacheSuite) TestPut(c *C) {
c.Assert(key, NotNil)
c.Assert(key, Equals, keys[i])

element, exists := lru.elements[string(keys[i].Hash())]
element, exists := lruMaxMem.elements[string(keys[i].Hash())]
c.Assert(exists, IsTrue)
c.Assert(element, NotNil)
c.Assert(element, Equals, root)
Expand Down
192 changes: 192 additions & 0 deletions util/stmtsummary/evicted.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,192 @@
// Copyright 2021 PingCAP, Inc.
//
// Licensed under the Apache License, Version 2.0 (the "License");
// you may not use this file except in compliance with the License.
// You may obtain a copy of the License at
//
// http://www.apache.org/licenses/LICENSE-2.0
//
// Unless required by applicable law or agreed to in writing, software
// distributed under the License is distributed on an "AS IS" BASIS,
// See the License for the specific language governing permissions and
// limitations under the License.

package stmtsummary

import (
"container/list"
"time"

"github.com/pingcap/parser/mysql"
"github.com/pingcap/tidb/types"
)

// stmtSummaryByDigestEvicted contents digests evicted from stmtSummaryByDigestMap
type stmtSummaryByDigestEvicted struct {
// record evicted data in intervals
// latest history data is Back()
history *list.List
}

// element being stored in stmtSummaryByDigestEvicted
type stmtSummaryByDigestEvictedElement struct {
// beginTime is the begin time of current interval
beginTime int64
// endTime is the end time of current interval
endTime int64
// *Kinds* of digest being evicted
digestKeyMap map[string]struct{}
}

// spawn a new pointer to stmtSummaryByDigestEvicted
func newStmtSummaryByDigestEvicted() *stmtSummaryByDigestEvicted {
return &stmtSummaryByDigestEvicted{
history: list.New(),
}
}

// spawn a new pointer to stmtSummaryByDigestEvictedElement
func newStmtSummaryByDigestEvictedElement(beginTime int64, endTime int64) *stmtSummaryByDigestEvictedElement {
return &stmtSummaryByDigestEvictedElement{
beginTime: beginTime,
endTime: endTime,
digestKeyMap: make(map[string]struct{}),
}
}

// AddEvicted is used add an evicted record to stmtSummaryByDigestEvicted
func (ssbde *stmtSummaryByDigestEvicted) AddEvicted(evictedKey *stmtSummaryByDigestKey, evictedValue *stmtSummaryByDigest, historySize int) {
if evictedValue == nil {
return
}

evictedValue.Lock()
defer evictedValue.Unlock()

if evictedValue.history == nil {
return
}
for e, h := evictedValue.history.Back(), ssbde.history.Back(); e != nil; e = e.Prev() {
evictedElement := e.Value.(*stmtSummaryByDigestElement)

// use closure to minimize time holding lock
func() {
evictedElement.Lock()
defer evictedElement.Unlock()
// no record in ssbde.history, direct insert
if ssbde.history.Len() == 0 && historySize != 0 {

eBeginTime := evictedElement.beginTime
eEndTime := evictedElement.endTime
record := newStmtSummaryByDigestEvictedElement(eBeginTime, eEndTime)
record.addEvicted(evictedKey, evictedElement)
ssbde.history.PushFront(record)

h = ssbde.history.Back()
return
}

// look for matching history interval
MATCHING:
for ; h != nil; h = h.Prev() {
historyElement := h.Value.(*stmtSummaryByDigestEvictedElement)

switch historyElement.matchAndAdd(evictedKey, evictedElement) {
case isMatch:
// automatically added
break MATCHING
// not matching, create a new record and insert
case isTooYoung:
{
eBeginTime := evictedElement.beginTime
eEndTime := evictedElement.endTime
record := newStmtSummaryByDigestEvictedElement(eBeginTime, eEndTime)
record.addEvicted(evictedKey, evictedElement)
ssbde.history.InsertAfter(record, h)
break MATCHING
}
default: // isTooOld
{
if h == ssbde.history.Front() {
// if digest older than all records in ssbde.history.
eBeginTime := evictedElement.beginTime
eEndTime := evictedElement.endTime
record := newStmtSummaryByDigestEvictedElement(eBeginTime, eEndTime)
record.addEvicted(evictedKey, evictedElement)
ssbde.history.PushFront(record)
break MATCHING
}
}
}
}
}()

// prevent exceeding history size
for ssbde.history.Len() > historySize && ssbde.history.Len() > 0 {
ssbde.history.Remove(ssbde.history.Front())
}
}
}

// Clear up all records in stmtSummaryByDigestEvicted
func (ssbde *stmtSummaryByDigestEvicted) Clear() {
ssbde.history.Init()
}

// add an evicted record to stmtSummaryByDigestEvictedElement
func (seElement *stmtSummaryByDigestEvictedElement) addEvicted(digestKey *stmtSummaryByDigestKey, digestValue *stmtSummaryByDigestElement) {
if digestKey != nil {
seElement.digestKeyMap[string(digestKey.Hash())] = struct{}{}
}
}

const (
isMatch = 0
isTooOld = 1
isTooYoung = 2
)

// matchAndAdd check time interval of seElement and digestValue.
// if matches, it will add the digest and return enum match
// if digest too old, it will return enum tooOld and do nothing
// if digest too young, it will return enum tooYoung and do nothing
func (seElement *stmtSummaryByDigestEvictedElement) matchAndAdd(digestKey *stmtSummaryByDigestKey, digestValue *stmtSummaryByDigestElement) (statement int) {
if seElement == nil || digestValue == nil {
return isTooYoung
}
sBeginTime, sEndTime := seElement.beginTime, seElement.endTime
eBeginTime, eEndTime := digestValue.beginTime, digestValue.endTime
if sBeginTime <= eBeginTime && eEndTime <= sEndTime {
seElement.addEvicted(digestKey, digestValue)
return isMatch
} else if eEndTime <= sBeginTime {
return isTooOld
} else {
return isTooYoung
}
}

// ToEvictedCountDatum converts history evicted record to `evicted count` record's datum
func (ssbde *stmtSummaryByDigestEvicted) ToEvictedCountDatum() [][]types.Datum {
records := make([][]types.Datum, 0, ssbde.history.Len())
for e := ssbde.history.Back(); e != nil; e = e.Prev() {
if record := e.Value.(*stmtSummaryByDigestEvictedElement).toEvictedCountDatum(); record != nil {
records = append(records, record)
}
}
return records
}

// toEvictedCountDatum converts evicted record to `EvictedCount` record's datum
func (seElement *stmtSummaryByDigestEvictedElement) toEvictedCountDatum() []types.Datum {
datum := types.MakeDatums(
types.NewTime(types.FromGoTime(time.Unix(seElement.beginTime, 0)), mysql.TypeTimestamp, 0),
types.NewTime(types.FromGoTime(time.Unix(seElement.endTime, 0)), mysql.TypeTimestamp, 0),
int64(len(seElement.digestKeyMap)),
)
return datum
}

func (ssMap *stmtSummaryByDigestMap) ToEvictedCountDatum() [][]types.Datum {
return ssMap.other.ToEvictedCountDatum()
}
Loading

0 comments on commit 0367c54

Please sign in to comment.