planner: improve OptimizationFlags to avoid internal sql's syncload (#61401)

This commit is contained in:
Weizhen Wang
2025-05-29 20:46:10 +08:00
committed by GitHub
parent 684010c999
commit 121cbcba2c
2 changed files with 7 additions and 8 deletions

View File

@ -382,8 +382,10 @@ func adjustOptimizationFlags(flag uint64, logic base.LogicalPlan) uint64 {
// When we use the straight Join Order hint, we should disable the join reorder optimization.
flag &= ^rule.FlagJoinReOrder
}
flag |= rule.FlagCollectPredicateColumnsPoint
flag |= rule.FlagSyncWaitStatsLoadPoint
if !logic.SCtx().GetSessionVars().InRestrictedSQL {
flag |= rule.FlagCollectPredicateColumnsPoint
flag |= rule.FlagSyncWaitStatsLoadPoint
}
if !logic.SCtx().GetSessionVars().StmtCtx.UseDynamicPruneMode {
flag |= rule.FlagPartitionProcessor // apply partition pruning under static mode
}

View File

@ -29,6 +29,7 @@ import (
"github.com/pingcap/tidb/pkg/sessionctx/vardef"
"github.com/pingcap/tidb/pkg/statistics"
"github.com/pingcap/tidb/pkg/statistics/asyncload"
"github.com/pingcap/tidb/pkg/util/intest"
"github.com/pingcap/tidb/pkg/util/intset"
"github.com/pingcap/tidb/pkg/util/logutil"
"go.uber.org/zap"
@ -40,9 +41,7 @@ type CollectPredicateColumnsPoint struct{}
// Optimize implements LogicalOptRule.<0th> interface.
func (c *CollectPredicateColumnsPoint) Optimize(_ context.Context, plan base.LogicalPlan, _ *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, bool, error) {
planChanged := false
if plan.SCtx().GetSessionVars().InRestrictedSQL {
return plan, planChanged, nil
}
intest.Assert(!plan.SCtx().GetSessionVars().InRestrictedSQL, "CollectPredicateColumnsPoint should not be called in restricted SQL mode")
syncWait := plan.SCtx().GetSessionVars().StatsLoadSyncWait.Load()
syncLoadEnabled := syncWait > 0
predicateColumns, visitedPhysTblIDs, tid2pids, opNum := CollectColumnStatsUsage(plan)
@ -225,9 +224,7 @@ type SyncWaitStatsLoadPoint struct{}
// Optimize implements the base.LogicalOptRule.<0th> interface.
func (SyncWaitStatsLoadPoint) Optimize(_ context.Context, plan base.LogicalPlan, _ *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, bool, error) {
planChanged := false
if plan.SCtx().GetSessionVars().InRestrictedSQL {
return plan, planChanged, nil
}
intest.Assert(!plan.SCtx().GetSessionVars().InRestrictedSQL, "SyncWaitStatsLoadPoint should not be called in restricted SQL mode")
if plan.SCtx().GetSessionVars().StmtCtx.IsSyncStatsFailed {
return plan, planChanged, nil
}