Skip to content

Commit

Permalink
don't load stats for point get index usage reporter
Browse files Browse the repository at this point in the history
Signed-off-by: Yang Keao <[email protected]>
  • Loading branch information
YangKeao committed Dec 23, 2024
1 parent ef7ade7 commit 14485c4
Show file tree
Hide file tree
Showing 4 changed files with 31 additions and 18 deletions.
24 changes: 13 additions & 11 deletions pkg/executor/builder.go
Original file line number Diff line number Diff line change
Expand Up @@ -3571,7 +3571,7 @@ func buildNoRangeTableReader(b *executorBuilder, v *plannercore.PhysicalTableRea
e := &TableReaderExecutor{
BaseExecutorV2: exec.NewBaseExecutorV2(b.ctx.GetSessionVars(), v.Schema(), v.ID()),
tableReaderExecutorContext: newTableReaderExecutorContext(b.ctx),
indexUsageReporter: b.buildIndexUsageReporter(v),
indexUsageReporter: b.buildIndexUsageReporter(v, true),
dagPB: dagReq,
startTS: startTS,
txnScope: b.txnScope,
Expand Down Expand Up @@ -3910,7 +3910,7 @@ func buildNoRangeIndexReader(b *executorBuilder, v *plannercore.PhysicalIndexRea
e := &IndexReaderExecutor{
indexReaderExecutorContext: newIndexReaderExecutorContext(b.ctx),
BaseExecutorV2: exec.NewBaseExecutorV2(b.ctx.GetSessionVars(), v.Schema(), v.ID()),
indexUsageReporter: b.buildIndexUsageReporter(v),
indexUsageReporter: b.buildIndexUsageReporter(v, true),
dagPB: dagReq,
startTS: startTS,
txnScope: b.txnScope,
Expand Down Expand Up @@ -4089,7 +4089,7 @@ func buildNoRangeIndexLookUpReader(b *executorBuilder, v *plannercore.PhysicalIn
e := &IndexLookUpExecutor{
indexLookUpExecutorContext: newIndexLookUpExecutorContext(b.ctx),
BaseExecutorV2: exec.NewBaseExecutorV2(b.ctx.GetSessionVars(), v.Schema(), v.ID()),
indexUsageReporter: b.buildIndexUsageReporter(v),
indexUsageReporter: b.buildIndexUsageReporter(v, true),
dagPB: indexReq,
startTS: startTS,
table: tbl,
Expand Down Expand Up @@ -4240,7 +4240,7 @@ func buildNoRangeIndexMergeReader(b *executorBuilder, v *plannercore.PhysicalInd

e := &IndexMergeReaderExecutor{
BaseExecutor: exec.NewBaseExecutor(b.ctx, v.Schema(), v.ID()),
indexUsageReporter: b.buildIndexUsageReporter(v),
indexUsageReporter: b.buildIndexUsageReporter(v, true),
dagPBs: partialReqs,
startTS: startTS,
table: tblInfo,
Expand Down Expand Up @@ -4272,13 +4272,15 @@ type tableStatsPreloader interface {
LoadTableStats(sessionctx.Context)
}

func buildIndexUsageReporter(ctx sessionctx.Context, plan tableStatsPreloader) (indexUsageReporter *exec.IndexUsageReporter) {
func buildIndexUsageReporter(ctx sessionctx.Context, plan tableStatsPreloader, loadStats bool) (indexUsageReporter *exec.IndexUsageReporter) {
sc := ctx.GetSessionVars().StmtCtx
if ctx.GetSessionVars().StmtCtx.IndexUsageCollector != nil &&
sc.RuntimeStatsColl != nil {
// Preload the table stats. If the statement is a point-get or execute, the planner may not have loaded the
// stats.
plan.LoadTableStats(ctx)
if loadStats {
// Preload the table stats. If the statement is a point-get or execute, the planner may not have loaded the
// stats.
plan.LoadTableStats(ctx)
}

statsMap := sc.GetUsedStatsInfo(false)
indexUsageReporter = exec.NewIndexUsageReporter(
Expand All @@ -4289,8 +4291,8 @@ func buildIndexUsageReporter(ctx sessionctx.Context, plan tableStatsPreloader) (
return indexUsageReporter
}

func (b *executorBuilder) buildIndexUsageReporter(plan tableStatsPreloader) (indexUsageReporter *exec.IndexUsageReporter) {
return buildIndexUsageReporter(b.ctx, plan)
func (b *executorBuilder) buildIndexUsageReporter(plan tableStatsPreloader, loadStats bool) (indexUsageReporter *exec.IndexUsageReporter) {
return buildIndexUsageReporter(b.ctx, plan, loadStats)
}

func (b *executorBuilder) buildIndexMergeReader(v *plannercore.PhysicalIndexMergeReader) exec.Executor {
Expand Down Expand Up @@ -5308,7 +5310,7 @@ func (b *executorBuilder) buildBatchPointGet(plan *plannercore.BatchPointGetPlan
decoder := NewRowDecoder(b.ctx, plan.Schema(), plan.TblInfo)
e := &BatchPointGetExec{
BaseExecutor: exec.NewBaseExecutor(b.ctx, plan.Schema(), plan.ID()),
indexUsageReporter: b.buildIndexUsageReporter(plan),
indexUsageReporter: b.buildIndexUsageReporter(plan, true),
tblInfo: plan.TblInfo,
idxInfo: plan.IndexInfo,
rowDecoder: decoder,
Expand Down
12 changes: 10 additions & 2 deletions pkg/executor/internal/exec/indexusage.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,8 @@
package exec

import (
"math"

"github.com/pingcap/tidb/pkg/meta/model"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/statistics"
Expand Down Expand Up @@ -101,8 +103,10 @@ func (e *IndexUsageReporter) ReportPointGetIndexUsageForHandle(tblInfo *model.Ta
func (e *IndexUsageReporter) ReportPointGetIndexUsage(tableID int64, physicalTableID int64, indexID int64, kvRequestTotal, rows int64) {
tableRowCount, ok := e.getTableRowCount(physicalTableID)
if !ok {
// skip if the table is empty or the stats is not valid
return
// it's possible that the point get doesn't have the table stats. In this case, we always
// report the tableRowCount as `math.MaxInt32`, so that it'll be recorded in the smallest
// non-zero bucket if the rows is greater than 0.
tableRowCount = math.MaxInt32
}

sample := indexusage.NewSample(0, uint64(kvRequestTotal), uint64(rows), uint64(tableRowCount))
Expand All @@ -111,6 +115,10 @@ func (e *IndexUsageReporter) ReportPointGetIndexUsage(tableID int64, physicalTab

// getTableRowCount returns the `RealtimeCount` of a table
func (e *IndexUsageReporter) getTableRowCount(tableID int64) (int64, bool) {
if e.statsMap == nil {
return 0, false
}

stats := e.statsMap.GetUsedInfo(tableID)
if stats == nil {
return 0, false
Expand Down
9 changes: 6 additions & 3 deletions pkg/executor/internal/exec/indexusage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -269,7 +269,8 @@ func TestIndexUsageReporterWithRealData(t *testing.T) {
"select * from t where id_1 = 1",
"Point_Get",
[]indexStatsExpect{
{tableID, idx1ID, []indexusage.Sample{indexusage.NewSample(1, 1, 1, 100)}},
// The point get will always use smallest bucket.
{tableID, idx1ID, []indexusage.Sample{indexusage.NewSample(1, 1, 1, 1000)}},
},
},
{
Expand Down Expand Up @@ -519,13 +520,15 @@ func TestIndexUsageReporterWithClusterIndex(t *testing.T) {
{
"select * from t0 where id = 1",
"Point_Get",
[]indexStatsExpect{{testTableInfos[0].tableID, testTableInfos[0].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 1, 100)}}},
// The point get will always use smallest bucket.
[]indexStatsExpect{{testTableInfos[0].tableID, testTableInfos[0].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 1, 1000)}}},
},
// PointGet on CommonHandle
{
"select * from t1 where id = \"1\"",
"Point_Get",
[]indexStatsExpect{{testTableInfos[1].tableID, testTableInfos[1].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 1, 100)}}},
// The point get will always use smallest bucket.
[]indexStatsExpect{{testTableInfos[1].tableID, testTableInfos[1].pkID, []indexusage.Sample{indexusage.NewSample(1, 1, 1, 1000)}}},
},
// BatchPointGet on PKAsHandle
{
Expand Down
4 changes: 2 additions & 2 deletions pkg/executor/point_get.go
Original file line number Diff line number Diff line change
Expand Up @@ -73,7 +73,7 @@ func (b *executorBuilder) buildPointGet(p *plannercore.PointGetPlan) exec.Execut

e := &PointGetExecutor{
BaseExecutor: exec.NewBaseExecutor(b.ctx, p.Schema(), p.ID()),
indexUsageReporter: b.buildIndexUsageReporter(p),
indexUsageReporter: b.buildIndexUsageReporter(p, false),
txnScope: b.txnScope,
readReplicaScope: b.readReplicaScope,
isStaleness: b.isStaleness,
Expand Down Expand Up @@ -204,7 +204,7 @@ func (e *PointGetExecutor) Recreated(p *plannercore.PointGetPlan) {
// It's necessary to at least reset the `runtimeStats` of the `BaseExecutor`.
// As the `StmtCtx` may have changed, a new index usage reporter should also be created.
e.BaseExecutor = exec.NewBaseExecutor(e.Ctx(), p.Schema(), p.ID())
e.indexUsageReporter = buildIndexUsageReporter(e.Ctx(), p)
e.indexUsageReporter = buildIndexUsageReporter(e.Ctx(), p, false)
}

// Init set fields needed for PointGetExecutor reuse, this does NOT change baseExecutor field
Expand Down

0 comments on commit 14485c4

Please sign in to comment.