92 lines
2.9 KiB
Go
92 lines
2.9 KiB
Go
// Copyright 2024 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,
|
|
// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
|
// See the License for the specific language governing permissions and
|
|
// limitations under the License.
|
|
|
|
package exec
|
|
|
|
import (
|
|
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
|
|
"github.com/pingcap/tidb/pkg/statistics"
|
|
"github.com/pingcap/tidb/pkg/statistics/handle/usage/indexusage"
|
|
"github.com/pingcap/tidb/pkg/util/execdetails"
|
|
)
|
|
|
|
// IndexUsageReporter is a toolkit to report index usage
|
|
type IndexUsageReporter struct {
|
|
reporter *indexusage.StmtIndexUsageCollector
|
|
runtimeStatsColl *execdetails.RuntimeStatsColl
|
|
statsMap map[int64]*stmtctx.UsedStatsInfoForTable
|
|
}
|
|
|
|
// NewIndexUsageReporter creates an index usage reporter util
|
|
func NewIndexUsageReporter(reporter *indexusage.StmtIndexUsageCollector,
|
|
runtimeStatsColl *execdetails.RuntimeStatsColl,
|
|
statsMap map[int64]*stmtctx.UsedStatsInfoForTable) *IndexUsageReporter {
|
|
return &IndexUsageReporter{
|
|
reporter: reporter,
|
|
runtimeStatsColl: runtimeStatsColl,
|
|
statsMap: statsMap,
|
|
}
|
|
}
|
|
|
|
// ReportCopIndexUsage reports the index usage to the inside collector
|
|
func (e *IndexUsageReporter) ReportCopIndexUsage(tableID int64, indexID int64, planID int) {
|
|
tableRowCount, ok := e.getTableRowCount(tableID)
|
|
if !ok {
|
|
// skip if the table is empty or the stats is not valid
|
|
return
|
|
}
|
|
|
|
copStats := e.runtimeStatsColl.GetCopStats(planID)
|
|
if copStats == nil {
|
|
return
|
|
}
|
|
copStats.Lock()
|
|
defer copStats.Unlock()
|
|
kvReq := copStats.GetTasks()
|
|
accessRows := copStats.GetActRows()
|
|
|
|
sample := indexusage.NewSample(0, uint64(kvReq), uint64(accessRows), uint64(tableRowCount))
|
|
e.reporter.Update(tableID, indexID, sample)
|
|
}
|
|
|
|
// ReportPointGetIndexUsage reports the index usage of a point get or batch point get
|
|
func (e *IndexUsageReporter) ReportPointGetIndexUsage(tableID int64, indexID int64, planID int, kvRequestTotal int64) {
|
|
tableRowCount, ok := e.getTableRowCount(tableID)
|
|
if !ok {
|
|
// skip if the table is empty or the stats is not valid
|
|
return
|
|
}
|
|
|
|
basic := e.runtimeStatsColl.GetBasicRuntimeStats(planID)
|
|
if basic == nil {
|
|
return
|
|
}
|
|
accessRows := basic.GetActRows()
|
|
|
|
sample := indexusage.NewSample(0, uint64(kvRequestTotal), uint64(accessRows), uint64(tableRowCount))
|
|
e.reporter.Update(tableID, indexID, sample)
|
|
}
|
|
|
|
// getTableRowCount returns the `RealtimeCount` of a table
|
|
func (e *IndexUsageReporter) getTableRowCount(tableID int64) (int64, bool) {
|
|
stats, ok := e.statsMap[tableID]
|
|
if !ok {
|
|
return 0, false
|
|
}
|
|
if stats.Version == statistics.PseudoVersion {
|
|
return 0, false
|
|
}
|
|
return stats.RealtimeCount, true
|
|
}
|