diff --git a/pkg/executor/builder.go b/pkg/executor/builder.go index f2541a2c365e4..72be28aec7253 100644 --- a/pkg/executor/builder.go +++ b/pkg/executor/builder.go @@ -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, @@ -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, @@ -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, @@ -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, @@ -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( @@ -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 { @@ -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, diff --git a/pkg/executor/internal/exec/indexusage.go b/pkg/executor/internal/exec/indexusage.go index addc20265323a..438403b73d8dc 100644 --- a/pkg/executor/internal/exec/indexusage.go +++ b/pkg/executor/internal/exec/indexusage.go @@ -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" @@ -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)) @@ -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 diff --git a/pkg/executor/internal/exec/indexusage_test.go b/pkg/executor/internal/exec/indexusage_test.go index 8ecf11ec6a23c..2c2f1ffaea720 100644 --- a/pkg/executor/internal/exec/indexusage_test.go +++ b/pkg/executor/internal/exec/indexusage_test.go @@ -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)}}, }, }, { @@ -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 { diff --git a/pkg/executor/point_get.go b/pkg/executor/point_get.go index fba16bc25d255..587fb9db9a1ad 100644 --- a/pkg/executor/point_get.go +++ b/pkg/executor/point_get.go @@ -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, @@ -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