Skip to content

Commit

Permalink
planner: move index_usage code into a separate package (#47163)
Browse files Browse the repository at this point in the history
ref #46905
  • Loading branch information
qw4990 authored Sep 22, 2023
1 parent 55ac974 commit 8fcbcd9
Show file tree
Hide file tree
Showing 7 changed files with 242 additions and 131 deletions.
1 change: 1 addition & 0 deletions session/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -61,6 +61,7 @@ go_library(
"//sessiontxn/isolation",
"//sessiontxn/staleread",
"//statistics/handle",
"//statistics/handle/usage",
"//store/driver/error",
"//store/driver/txn",
"//store/helper",
Expand Down
5 changes: 3 additions & 2 deletions session/session.go
Original file line number Diff line number Diff line change
Expand Up @@ -77,6 +77,7 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/sessiontxn"
"github.com/pingcap/tidb/statistics/handle"
"github.com/pingcap/tidb/statistics/handle/usage"
storeerr "github.com/pingcap/tidb/store/driver/error"
"github.com/pingcap/tidb/store/driver/txn"
"github.com/pingcap/tidb/store/helper"
Expand Down Expand Up @@ -239,7 +240,7 @@ type session struct {
mppClient kv.MPPClient

// indexUsageCollector collects index usage information.
idxUsageCollector *handle.SessionIndexUsageCollector
idxUsageCollector *usage.SessionIndexUsageCollector

functionUsageMu struct {
syncutil.RWMutex
Expand Down Expand Up @@ -465,7 +466,7 @@ func (s *session) StoreIndexUsage(tblID int64, idxID int64, rowsSelected int64)
if s.idxUsageCollector == nil {
return
}
s.idxUsageCollector.Update(tblID, idxID, &handle.IndexUsageInformation{QueryCount: 1, RowsSelected: rowsSelected})
s.idxUsageCollector.Update(tblID, idxID, &usage.IndexUsageInformation{QueryCount: 1, RowsSelected: rowsSelected})
}

// FieldList returns fields list of a table.
Expand Down
1 change: 1 addition & 0 deletions statistics/handle/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -34,6 +34,7 @@ go_library(
"//statistics/handle/globalstats",
"//statistics/handle/lockstats",
"//statistics/handle/metrics",
"//statistics/handle/usage",
"//table",
"//types",
"//util",
Expand Down
5 changes: 3 additions & 2 deletions statistics/handle/handle.go
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,7 @@ import (
"github.com/pingcap/tidb/statistics/handle/cache"
"github.com/pingcap/tidb/statistics/handle/globalstats"
handle_metrics "github.com/pingcap/tidb/statistics/handle/metrics"
"github.com/pingcap/tidb/statistics/handle/usage"
"github.com/pingcap/tidb/table"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util/chunk"
Expand Down Expand Up @@ -76,7 +77,7 @@ type Handle struct {
ddlEventCh chan *ddlUtil.Event

// idxUsageListHead contains all the index usage collectors required by session.
idxUsageListHead *SessionIndexUsageCollector
idxUsageListHead *usage.SessionIndexUsageCollector

// listHead contains all the stats collector required by session.
listHead *SessionStatsCollector
Expand Down Expand Up @@ -181,7 +182,7 @@ func NewHandle(_, initStatsCtx sessionctx.Context, lease time.Duration, pool ses
gpool: gp.New(math.MaxInt16, time.Minute),
ddlEventCh: make(chan *ddlUtil.Event, 1000),
listHead: NewSessionStatsCollector(),
idxUsageListHead: &SessionIndexUsageCollector{mapper: make(indexUsageMap)},
idxUsageListHead: usage.NewSessionIndexUsageCollector(nil),
pool: pool,
sysProcTracker: tracker,
autoAnalyzeProcIDGetter: autoAnalyzeProcIDGetter,
Expand Down
150 changes: 23 additions & 127 deletions statistics/handle/update.go
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ import (
"github.com/pingcap/tidb/sessionctx/variable"
"github.com/pingcap/tidb/statistics"
"github.com/pingcap/tidb/statistics/handle/cache"
"github.com/pingcap/tidb/statistics/handle/usage"
"github.com/pingcap/tidb/types"
"github.com/pingcap/tidb/util"
"github.com/pingcap/tidb/util/logutil"
Expand Down Expand Up @@ -210,102 +211,11 @@ func (h *Handle) NewSessionStatsCollector() *SessionStatsCollector {
return newCollector
}

// IndexUsageInformation is the data struct to store index usage information.
type IndexUsageInformation struct {
LastUsedAt string
QueryCount int64
RowsSelected int64
}

// GlobalIndexID is the key type for indexUsageMap.
type GlobalIndexID struct {
TableID int64
IndexID int64
}

type indexUsageMap map[GlobalIndexID]IndexUsageInformation

// SessionIndexUsageCollector is a list item that holds the index usage mapper. If you want to write or read mapper, you must lock it.
type SessionIndexUsageCollector struct {
mapper indexUsageMap
next *SessionIndexUsageCollector
sync.Mutex

deleted bool
}

func (m indexUsageMap) updateByKey(id GlobalIndexID, value *IndexUsageInformation) {
item := m[id]
item.QueryCount += value.QueryCount
item.RowsSelected += value.RowsSelected
if item.LastUsedAt < value.LastUsedAt {
item.LastUsedAt = value.LastUsedAt
}
m[id] = item
}

func (m indexUsageMap) update(tableID int64, indexID int64, value *IndexUsageInformation) {
id := GlobalIndexID{TableID: tableID, IndexID: indexID}
m.updateByKey(id, value)
}

func (m indexUsageMap) merge(destMap indexUsageMap) {
for id := range destMap {
item := destMap[id]
m.updateByKey(id, &item)
}
}

// Update updates the mapper in SessionIndexUsageCollector.
func (s *SessionIndexUsageCollector) Update(tableID int64, indexID int64, value *IndexUsageInformation) {
value.LastUsedAt = time.Now().Format(types.TimeFSPFormat)
s.Lock()
defer s.Unlock()
s.mapper.update(tableID, indexID, value)
}

// Delete will set s.deleted to true which means it can be deleted from linked list.
func (s *SessionIndexUsageCollector) Delete() {
s.Lock()
defer s.Unlock()
s.deleted = true
}

// NewSessionIndexUsageCollector will add a new SessionIndexUsageCollector into linked list headed by idxUsageListHead.
// idxUsageListHead always points to an empty SessionIndexUsageCollector as a sentinel node. So we let idxUsageListHead.next
// points to new item. It's helpful to sweepIdxUsageList.
func (h *Handle) NewSessionIndexUsageCollector() *SessionIndexUsageCollector {
h.idxUsageListHead.Lock()
defer h.idxUsageListHead.Unlock()
newCollector := &SessionIndexUsageCollector{
mapper: make(indexUsageMap),
next: h.idxUsageListHead.next,
}
h.idxUsageListHead.next = newCollector
return newCollector
}

// sweepIdxUsageList will loop over the list, merge each session's local index usage information into handle
// and remove closed session's collector.
// For convenience, we keep idxUsageListHead always points to sentinel node. So that we don't need to consider corner case.
func (h *Handle) sweepIdxUsageList() indexUsageMap {
prev := h.idxUsageListHead
prev.Lock()
mapper := make(indexUsageMap)
for curr := prev.next; curr != nil; curr = curr.next {
curr.Lock()
mapper.merge(curr.mapper)
if curr.deleted {
prev.next = curr.next
curr.Unlock()
} else {
prev.Unlock()
curr.mapper = make(indexUsageMap)
prev = curr
}
}
prev.Unlock()
return mapper
func (h *Handle) NewSessionIndexUsageCollector() *usage.SessionIndexUsageCollector {
return usage.NewSessionIndexUsageCollector(h.idxUsageListHead)
}

// batchInsertSize is the batch size used by internal SQL to insert values to some system table.
Expand All @@ -316,47 +226,33 @@ const maxInsertLength = 1024 * 1024

// DumpIndexUsageToKV will dump in-memory index usage information to KV.
func (h *Handle) DumpIndexUsageToKV() error {
ctx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnStats)
mapper := h.sweepIdxUsageList()
type FullIndexUsageInformation struct {
information IndexUsageInformation
id GlobalIndexID
}
indexInformationSlice := make([]FullIndexUsageInformation, 0, len(mapper))
for id, value := range mapper {
indexInformationSlice = append(indexInformationSlice, FullIndexUsageInformation{id: id, information: value})
return h.callWithExec(func(exec sqlexec.RestrictedSQLExecutor) error {
return usage.DumpIndexUsageToKV(exec, h.idxUsageListHead)
})
}

func (h *Handle) callWithExec(f func(exec sqlexec.RestrictedSQLExecutor) error) (err error) {
se, err := h.pool.Get()
if err != nil {
return err
}
for i := 0; i < len(mapper); i += batchInsertSize {
end := i + batchInsertSize
if end > len(mapper) {
end = len(mapper)
}
sql := new(strings.Builder)
sqlexec.MustFormatSQL(sql, "insert into mysql.SCHEMA_INDEX_USAGE (table_id,index_id,query_count,rows_selected,last_used_at) values")
for j := i; j < end; j++ {
index := indexInformationSlice[j]
sqlexec.MustFormatSQL(sql, "(%?, %?, %?, %?, %?)", index.id.TableID, index.id.IndexID,
index.information.QueryCount, index.information.RowsSelected, index.information.LastUsedAt)
if j < end-1 {
sqlexec.MustFormatSQL(sql, ",")
}
}
sqlexec.MustFormatSQL(sql, "on duplicate key update query_count=query_count+values(query_count),rows_selected=rows_selected+values(rows_selected),last_used_at=greatest(last_used_at, values(last_used_at))")
if _, _, err := h.execRestrictedSQL(ctx, sql.String()); err != nil {
return errors.Trace(err)
defer func() {
if err == nil { // only recycle when no error
h.pool.Put(se)
}
}()
sctx := se.(sessionctx.Context)
if err := UpdateSCtxVarsForStats(sctx); err != nil { // update stats variables automatically
return err
}
return nil

exec := se.(sqlexec.RestrictedSQLExecutor)
return f(exec)
}

// GCIndexUsage will delete the usage information of those indexes that do not exist.
func (h *Handle) GCIndexUsage() error {
// For performance and implementation reasons, mysql.schema_index_usage doesn't handle DDL.
// We periodically delete the usage information of non-existent indexes through information_schema.tidb_indexes.
// This sql will delete the usage information of those indexes that not in information_schema.tidb_indexes.
sql := `delete from mysql.SCHEMA_INDEX_USAGE as stats where stats.index_id not in (select idx.index_id from information_schema.tidb_indexes as idx)`
_, _, err := h.execRestrictedSQL(context.Background(), sql)
return err
return h.callWithExec(usage.GCIndexUsageOnKV)
}

var (
Expand Down
14 changes: 14 additions & 0 deletions statistics/handle/usage/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -0,0 +1,14 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library")

go_library(
name = "usage",
srcs = ["index_usage.go"],
importpath = "github.com/pingcap/tidb/statistics/handle/usage",
visibility = ["//visibility:public"],
deps = [
"//kv",
"//types",
"//util/sqlexec",
"@com_github_pingcap_errors//:errors",
],
)
Loading

0 comments on commit 8fcbcd9

Please sign in to comment.