Skip to content

Commit

Permalink
planner: move more hint code to a separate package (#49697)
Browse files Browse the repository at this point in the history
ref #48875
  • Loading branch information
qw4990 authored Dec 22, 2023
1 parent 6c4eba0 commit ebd259c
Show file tree
Hide file tree
Showing 32 changed files with 672 additions and 592 deletions.
2 changes: 1 addition & 1 deletion pkg/executor/infoschema_reader.go
Original file line number Diff line number Diff line change
Expand Up @@ -738,7 +738,7 @@ func (e *hugeMemTableRetriever) dataForColumnsInTable(ctx context.Context, sctx
internalCtx := kv.WithInternalSourceType(context.Background(), kv.InternalTxnOthers)
// Build plan is not thread safe, there will be concurrency on sessionctx.
if err := runWithSystemSession(internalCtx, sctx, func(s sessionctx.Context) error {
planBuilder, _ := plannercore.NewPlanBuilder().Init(s, is, &hint.BlockHintProcessor{})
planBuilder, _ := plannercore.NewPlanBuilder().Init(s, is, &hint.QBHintHandler{})
var err error
viewLogicalPlan, err = planBuilder.BuildDataSourceFromView(ctx, schema.Name, tbl, nil, nil)
return errors.Trace(err)
Expand Down
2 changes: 1 addition & 1 deletion pkg/executor/show.go
Original file line number Diff line number Diff line change
Expand Up @@ -2359,7 +2359,7 @@ func tryFillViewColumnType(ctx context.Context, sctx sessionctx.Context, is info
return runWithSystemSession(ctx, sctx, func(s sessionctx.Context) error {
// Retrieve view columns info.
planBuilder, _ := plannercore.NewPlanBuilder(
plannercore.PlanBuilderOptNoExecution{}).Init(s, is, &hint.BlockHintProcessor{})
plannercore.PlanBuilderOptNoExecution{}).Init(s, is, &hint.QBHintHandler{})
viewLogicalPlan, err := planBuilder.BuildDataSourceFromView(ctx, dbName, tbl, nil, nil)
if err != nil {
return err
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/casetest/stats_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ func TestGroupNDVs(t *testing.T) {
err = core.Preprocess(context.Background(), tk.Session(), stmt, core.WithPreprocessorReturn(ret))
require.NoError(t, err)
tk.Session().GetSessionVars().PlanColumnID.Store(0)
builder, _ := core.NewPlanBuilder().Init(tk.Session(), ret.InfoSchema, &hint.BlockHintProcessor{})
builder, _ := core.NewPlanBuilder().Init(tk.Session(), ret.InfoSchema, &hint.QBHintHandler{})
p, err := builder.Build(ctx, stmt)
require.NoError(t, err, comment)
p, err = core.LogicalOptimizeTest(ctx, builder.GetOptFlag(), p.(core.LogicalPlan))
Expand Down
4 changes: 2 additions & 2 deletions pkg/planner/core/collect_column_stats_usage_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -259,7 +259,7 @@ func TestCollectPredicateColumns(t *testing.T) {
require.NoError(t, err, comment)
err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is}))
require.NoError(t, err, comment)
builder, _ := NewPlanBuilder().Init(s.ctx, s.is, &hint.BlockHintProcessor{})
builder, _ := NewPlanBuilder().Init(s.ctx, s.is, &hint.QBHintHandler{})
p, err := builder.Build(ctx, stmt)
require.NoError(t, err, comment)
lp, ok := p.(LogicalPlan)
Expand Down Expand Up @@ -342,7 +342,7 @@ func TestCollectHistNeededColumns(t *testing.T) {
require.NoError(t, err, comment)
err = Preprocess(context.Background(), s.ctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: s.is}))
require.NoError(t, err, comment)
builder, _ := NewPlanBuilder().Init(s.ctx, s.is, &hint.BlockHintProcessor{})
builder, _ := NewPlanBuilder().Init(s.ctx, s.is, &hint.QBHintHandler{})
p, err := builder.Build(ctx, stmt)
require.NoError(t, err, comment)
lp, ok := p.(LogicalPlan)
Expand Down
84 changes: 42 additions & 42 deletions pkg/planner/core/exhaust_physical_plans.go
Original file line number Diff line number Diff line change
Expand Up @@ -233,8 +233,8 @@ func (p *LogicalJoin) GetMergeJoin(prop *property.PhysicalProperty, schema *expr
}
}

if p.preferJoinType&preferNoMergeJoin > 0 {
if p.preferJoinType&preferMergeJoin == 0 {
if p.preferJoinType&h.PreferNoMergeJoin > 0 {
if p.preferJoinType&h.PreferMergeJoin == 0 {
return nil
}
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen(
Expand All @@ -243,7 +243,7 @@ func (p *LogicalJoin) GetMergeJoin(prop *property.PhysicalProperty, schema *expr

// If TiDB_SMJ hint is existed, it should consider enforce merge join,
// because we can't trust lhsChildProperty completely.
if (p.preferJoinType&preferMergeJoin) > 0 ||
if (p.preferJoinType&h.PreferMergeJoin) > 0 ||
p.shouldSkipHashJoin() { // if hash join is not allowed, generate as many other types of join as possible to avoid 'cant-find-plan' error.
joins = append(joins, p.getEnforcedMergeJoin(prop, schema, statsInfo)...)
}
Expand Down Expand Up @@ -392,15 +392,15 @@ var ForceUseOuterBuild4Test = atomic.NewBool(false)
var ForcedHashLeftJoin4Test = atomic.NewBool(false)

func (p *LogicalJoin) shouldSkipHashJoin() bool {
return (p.preferJoinType&preferNoHashJoin) > 0 || (p.SCtx().GetSessionVars().DisableHashJoin)
return (p.preferJoinType&h.PreferNoHashJoin) > 0 || (p.SCtx().GetSessionVars().DisableHashJoin)
}

func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []PhysicalPlan, forced bool) {
if !prop.IsSortItemEmpty() { // hash join doesn't promise any orders
return
}
forceLeftToBuild := ((p.preferJoinType & preferLeftAsHJBuild) > 0) || ((p.preferJoinType & preferRightAsHJProbe) > 0)
forceRightToBuild := ((p.preferJoinType & preferRightAsHJBuild) > 0) || ((p.preferJoinType & preferLeftAsHJProbe) > 0)
forceLeftToBuild := ((p.preferJoinType & h.PreferLeftAsHJBuild) > 0) || ((p.preferJoinType & h.PreferRightAsHJProbe) > 0)
forceRightToBuild := ((p.preferJoinType & h.PreferRightAsHJBuild) > 0) || ((p.preferJoinType & h.PreferLeftAsHJProbe) > 0)
if forceLeftToBuild && forceRightToBuild {
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints"))
forceLeftToBuild = false
Expand Down Expand Up @@ -454,7 +454,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy
}
}

forced = (p.preferJoinType&preferHashJoin > 0) || forceLeftToBuild || forceRightToBuild
forced = (p.preferJoinType&h.PreferHashJoin > 0) || forceLeftToBuild || forceRightToBuild
if !forced && p.shouldSkipHashJoin() {
return nil, false
} else if forced && p.shouldSkipHashJoin() {
Expand Down Expand Up @@ -604,7 +604,7 @@ func (p *LogicalJoin) constructIndexMergeJoin(
compareFilters *ColWithCmpFuncManager,
) []PhysicalPlan {
hintExists := false
if (outerIdx == 1 && (p.preferJoinType&preferLeftAsINLMJInner) > 0) || (outerIdx == 0 && (p.preferJoinType&preferRightAsINLMJInner) > 0) {
if (outerIdx == 1 && (p.preferJoinType&h.PreferLeftAsINLMJInner) > 0) || (outerIdx == 0 && (p.preferJoinType&h.PreferRightAsINLMJInner) > 0) {
hintExists = true
}
indexJoins := p.constructIndexJoin(prop, outerIdx, innerTask, ranges, keyOff2IdxOff, path, compareFilters, !hintExists)
Expand Down Expand Up @@ -782,7 +782,7 @@ func (p *LogicalJoin) extractIndexJoinInnerChildPattern(innerChild LogicalPlan)
wrapper.ds = ds
// If one of the union scan children is a TiFlash table, then we can't choose index join.
for _, child := range wrapper.us.Children() {
if ds, ok := child.(*DataSource); ok && ds.preferStoreType&preferTiFlash != 0 {
if ds, ok := child.(*DataSource); ok && ds.preferStoreType&h.PreferTiFlash != 0 {
return nil
}
}
Expand Down Expand Up @@ -813,7 +813,7 @@ func (p *LogicalJoin) extractIndexJoinInnerChildPattern(innerChild LogicalPlan)
}
wrapper.ds = ds
}
if wrapper.ds == nil || wrapper.ds.preferStoreType&preferTiFlash != 0 {
if wrapper.ds == nil || wrapper.ds.preferStoreType&h.PreferTiFlash != 0 {
return nil
}
return wrapper
Expand Down Expand Up @@ -2039,13 +2039,13 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ
// The priority is: force hints like TIDB_INLJ > filter hints like NO_INDEX_JOIN > variables.
// Handle hints conflict first.
stmtCtx := p.SCtx().GetSessionVars().StmtCtx
if p.preferAny(preferLeftAsINLJInner, preferRightAsINLJInner) && p.preferAny(preferNoIndexJoin) {
if p.preferAny(h.PreferLeftAsINLJInner, h.PreferRightAsINLJInner) && p.preferAny(h.PreferNoIndexJoin) {
stmtCtx.AppendWarning(ErrInternal.FastGen("Some INL_JOIN and NO_INDEX_JOIN hints conflict, NO_INDEX_JOIN may be ignored"))
}
if p.preferAny(preferLeftAsINLHJInner, preferRightAsINLHJInner) && p.preferAny(preferNoIndexHashJoin) {
if p.preferAny(h.PreferLeftAsINLHJInner, h.PreferRightAsINLHJInner) && p.preferAny(h.PreferNoIndexHashJoin) {
stmtCtx.AppendWarning(ErrInternal.FastGen("Some INL_HASH_JOIN and NO_INDEX_HASH_JOIN hints conflict, NO_INDEX_HASH_JOIN may be ignored"))
}
if p.preferAny(preferLeftAsINLMJInner, preferRightAsINLMJInner) && p.preferAny(preferNoIndexMergeJoin) {
if p.preferAny(h.PreferLeftAsINLMJInner, h.PreferRightAsINLMJInner) && p.preferAny(h.PreferNoIndexMergeJoin) {
stmtCtx.AppendWarning(ErrInternal.FastGen("Some INL_MERGE_JOIN and NO_INDEX_MERGE_JOIN hints conflict, NO_INDEX_MERGE_JOIN may be ignored"))
}

Expand All @@ -2058,7 +2058,7 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ
}

func (p *LogicalJoin) handleFilterIndexJoinHints(candidates []PhysicalPlan) []PhysicalPlan {
if !p.preferAny(preferNoIndexJoin, preferNoIndexHashJoin, preferNoIndexMergeJoin) {
if !p.preferAny(h.PreferNoIndexJoin, h.PreferNoIndexHashJoin, h.PreferNoIndexMergeJoin) {
return candidates // no filter index join hints
}
filtered := make([]PhysicalPlan, 0, len(candidates))
Expand All @@ -2067,9 +2067,9 @@ func (p *LogicalJoin) handleFilterIndexJoinHints(candidates []PhysicalPlan) []Ph
if !ok {
continue
}
if (p.preferAny(preferNoIndexJoin) && joinMethod == indexJoinMethod) ||
(p.preferAny(preferNoIndexHashJoin) && joinMethod == indexHashJoinMethod) ||
(p.preferAny(preferNoIndexMergeJoin) && joinMethod == indexMergeJoinMethod) {
if (p.preferAny(h.PreferNoIndexJoin) && joinMethod == indexJoinMethod) ||
(p.preferAny(h.PreferNoIndexHashJoin) && joinMethod == indexHashJoinMethod) ||
(p.preferAny(h.PreferNoIndexMergeJoin) && joinMethod == indexMergeJoinMethod) {
continue
}
filtered = append(filtered, candidate)
Expand All @@ -2079,8 +2079,8 @@ func (p *LogicalJoin) handleFilterIndexJoinHints(candidates []PhysicalPlan) []Ph

// handleForceIndexJoinHints handles the force index join hints and returns all plans that can satisfy the hints.
func (p *LogicalJoin) handleForceIndexJoinHints(prop *property.PhysicalProperty, candidates []PhysicalPlan) (indexJoins []PhysicalPlan, canForced bool) {
if !p.preferAny(preferRightAsINLJInner, preferRightAsINLHJInner, preferRightAsINLMJInner,
preferLeftAsINLJInner, preferLeftAsINLHJInner, preferLeftAsINLMJInner) {
if !p.preferAny(h.PreferRightAsINLJInner, h.PreferRightAsINLHJInner, h.PreferRightAsINLMJInner,
h.PreferLeftAsINLJInner, h.PreferLeftAsINLHJInner, h.PreferLeftAsINLMJInner) {
return candidates, false // no force index join hints
}
forced := make([]PhysicalPlan, 0, len(candidates))
Expand All @@ -2089,12 +2089,12 @@ func (p *LogicalJoin) handleForceIndexJoinHints(prop *property.PhysicalProperty,
if !ok {
continue
}
if (p.preferAny(preferLeftAsINLJInner) && innerSide == joinLeft && joinMethod == indexJoinMethod) ||
(p.preferAny(preferRightAsINLJInner) && innerSide == joinRight && joinMethod == indexJoinMethod) ||
(p.preferAny(preferLeftAsINLHJInner) && innerSide == joinLeft && joinMethod == indexHashJoinMethod) ||
(p.preferAny(preferRightAsINLHJInner) && innerSide == joinRight && joinMethod == indexHashJoinMethod) ||
(p.preferAny(preferLeftAsINLMJInner) && innerSide == joinLeft && joinMethod == indexMergeJoinMethod) ||
(p.preferAny(preferRightAsINLMJInner) && innerSide == joinRight && joinMethod == indexMergeJoinMethod) {
if (p.preferAny(h.PreferLeftAsINLJInner) && innerSide == joinLeft && joinMethod == indexJoinMethod) ||
(p.preferAny(h.PreferRightAsINLJInner) && innerSide == joinRight && joinMethod == indexJoinMethod) ||
(p.preferAny(h.PreferLeftAsINLHJInner) && innerSide == joinLeft && joinMethod == indexHashJoinMethod) ||
(p.preferAny(h.PreferRightAsINLHJInner) && innerSide == joinRight && joinMethod == indexHashJoinMethod) ||
(p.preferAny(h.PreferLeftAsINLMJInner) && innerSide == joinLeft && joinMethod == indexMergeJoinMethod) ||
(p.preferAny(h.PreferRightAsINLMJInner) && innerSide == joinRight && joinMethod == indexMergeJoinMethod) {
forced = append(forced, candidate)
}
}
Expand All @@ -2114,11 +2114,11 @@ func (p *LogicalJoin) handleForceIndexJoinHints(prop *property.PhysicalProperty,
}
var errMsg string
switch {
case p.preferAny(preferLeftAsINLJInner, preferRightAsINLJInner): // prefer index join
case p.preferAny(h.PreferLeftAsINLJInner, h.PreferRightAsINLJInner): // prefer index join
errMsg = fmt.Sprintf("Optimizer Hint %s or %s is inapplicable", h.Restore2JoinHint(h.HintINLJ, indexJoinTables), h.Restore2JoinHint(h.TiDBIndexNestedLoopJoin, indexJoinTables))
case p.preferAny(preferLeftAsINLHJInner, preferRightAsINLHJInner): // prefer index hash join
case p.preferAny(h.PreferLeftAsINLHJInner, h.PreferRightAsINLHJInner): // prefer index hash join
errMsg = fmt.Sprintf("Optimizer Hint %s is inapplicable", h.Restore2JoinHint(h.HintINLHJ, indexHashJoinTables))
case p.preferAny(preferLeftAsINLMJInner, preferRightAsINLMJInner): // prefer index merge join
case p.preferAny(h.PreferLeftAsINLMJInner, h.PreferRightAsINLMJInner): // prefer index merge join
errMsg = fmt.Sprintf("Optimizer Hint %s is inapplicable", h.Restore2JoinHint(h.HintINLMJ, indexMergeJoinTables))
}
// Append inapplicable reason.
Expand Down Expand Up @@ -2286,12 +2286,12 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P
joins := make([]PhysicalPlan, 0, 8)
canPushToTiFlash := p.canPushToCop(kv.TiFlash)
if p.SCtx().GetSessionVars().IsMPPAllowed() && canPushToTiFlash {
if (p.preferJoinType & preferShuffleJoin) > 0 {
if (p.preferJoinType & h.PreferShuffleJoin) > 0 {
if shuffleJoins := p.tryToGetMppHashJoin(prop, false); len(shuffleJoins) > 0 {
return shuffleJoins, true, nil
}
}
if (p.preferJoinType & preferBCJoin) > 0 {
if (p.preferJoinType & h.PreferBCJoin) > 0 {
if bcastJoins := p.tryToGetMppHashJoin(prop, true); len(bcastJoins) > 0 {
return bcastJoins, true, nil
}
Expand All @@ -2306,11 +2306,11 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P
} else {
hasMppHints := false
var errMsg string
if (p.preferJoinType & preferShuffleJoin) > 0 {
if (p.preferJoinType & h.PreferShuffleJoin) > 0 {
errMsg = "The join can not push down to the MPP side, the shuffle_join() hint is invalid"
hasMppHints = true
}
if (p.preferJoinType & preferBCJoin) > 0 {
if (p.preferJoinType & h.PreferBCJoin) > 0 {
errMsg = "The join can not push down to the MPP side, the broadcast_join() hint is invalid"
hasMppHints = true
}
Expand All @@ -2325,7 +2325,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P
if !p.isNAAJ() {
// naaj refuse merge join and index join.
mergeJoins := p.GetMergeJoin(prop, p.schema, p.StatsInfo(), p.children[0].StatsInfo(), p.children[1].StatsInfo())
if (p.preferJoinType&preferMergeJoin) > 0 && len(mergeJoins) > 0 {
if (p.preferJoinType&h.PreferMergeJoin) > 0 && len(mergeJoins) > 0 {
return mergeJoins, true, nil
}
joins = append(joins, mergeJoins...)
Expand Down Expand Up @@ -2434,8 +2434,8 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC
RightNAJoinKeys: rNAKeys,
}
// It indicates which side is the build side.
forceLeftToBuild := ((p.preferJoinType & preferLeftAsHJBuild) > 0) || ((p.preferJoinType & preferRightAsHJProbe) > 0)
forceRightToBuild := ((p.preferJoinType & preferRightAsHJBuild) > 0) || ((p.preferJoinType & preferLeftAsHJProbe) > 0)
forceLeftToBuild := ((p.preferJoinType & h.PreferLeftAsHJBuild) > 0) || ((p.preferJoinType & h.PreferRightAsHJProbe) > 0)
forceRightToBuild := ((p.preferJoinType & h.PreferRightAsHJBuild) > 0) || ((p.preferJoinType & h.PreferLeftAsHJProbe) > 0)
if forceLeftToBuild && forceRightToBuild {
p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.FastGen("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints"))
forceLeftToBuild = false
Expand Down Expand Up @@ -3193,7 +3193,7 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P
}
// If STREAM_AGG hint is existed, it should consider enforce stream aggregation,
// because we can't trust possibleChildProperty completely.
if (la.aggHints.PreferAggType & preferStreamAgg) > 0 {
if (la.aggHints.PreferAggType & h.PreferStreamAgg) > 0 {
streamAggs = append(streamAggs, la.getEnforcedStreamAggs(prop)...)
}
return streamAggs
Expand Down Expand Up @@ -3327,9 +3327,9 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert
// handle MPP Agg hints
var preferMode AggMppRunMode
var prefer bool
if la.aggHints.PreferAggType&preferMPP1PhaseAgg > 0 {
if la.aggHints.PreferAggType&h.PreferMPP1PhaseAgg > 0 {
preferMode, prefer = Mpp1Phase, true
} else if la.aggHints.PreferAggType&preferMPP2PhaseAgg > 0 {
} else if la.aggHints.PreferAggType&h.PreferMPP2PhaseAgg > 0 {
preferMode, prefer = Mpp2Phase, true
}
if prefer {
Expand Down Expand Up @@ -3382,11 +3382,11 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy
} else {
hasMppHints := false
var errMsg string
if la.aggHints.PreferAggType&preferMPP1PhaseAgg > 0 {
if la.aggHints.PreferAggType&h.PreferMPP1PhaseAgg > 0 {
errMsg = "The agg can not push down to the MPP side, the MPP_1PHASE_AGG() hint is invalid"
hasMppHints = true
}
if la.aggHints.PreferAggType&preferMPP2PhaseAgg > 0 {
if la.aggHints.PreferAggType&h.PreferMPP2PhaseAgg > 0 {
errMsg = "The agg can not push down to the MPP side, the MPP_2PHASE_AGG() hint is invalid"
hasMppHints = true
}
Expand Down Expand Up @@ -3416,8 +3416,8 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy
// ResetHintIfConflicted resets the aggHints.PreferAggType if they are conflicted,
// and returns the two PreferAggType hints.
func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferStream bool) {
preferHash = (la.aggHints.PreferAggType & preferHashAgg) > 0
preferStream = (la.aggHints.PreferAggType & preferStreamAgg) > 0
preferHash = (la.aggHints.PreferAggType & h.PreferHashAgg) > 0
preferStream = (la.aggHints.PreferAggType & h.PreferStreamAgg) > 0
if preferHash && preferStream {
errMsg := "Optimizer aggregation hints are conflicted"
warning := ErrInternal.FastGen(errMsg)
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/expression_rewriter.go
Original file line number Diff line number Diff line change
Expand Up @@ -68,7 +68,7 @@ func rewriteAstExpr(sctx sessionctx.Context, expr ast.ExprNode, schema *expressi
if s, ok := sctx.GetInfoSchema().(infoschema.InfoSchema); ok {
is = s
}
b, savedBlockNames := NewPlanBuilder().Init(sctx, is, &hint.BlockHintProcessor{})
b, savedBlockNames := NewPlanBuilder().Init(sctx, is, &hint.QBHintHandler{})
b.allowBuildCastArray = allowCastArray
fakePlan := LogicalTableDual{}.Init(sctx, 0)
if schema != nil {
Expand Down
7 changes: 4 additions & 3 deletions pkg/planner/core/find_best_task.go
Original file line number Diff line number Diff line change
Expand Up @@ -38,6 +38,7 @@ import (
tidbutil "github.com/pingcap/tidb/pkg/util"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/collate"
h "github.com/pingcap/tidb/pkg/util/hint"
"github.com/pingcap/tidb/pkg/util/logutil"
"github.com/pingcap/tidb/pkg/util/ranger"
"github.com/pingcap/tidb/pkg/util/tracing"
Expand Down Expand Up @@ -1257,10 +1258,10 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter
}
}
if path.IsTablePath() {
if ds.preferStoreType&preferTiFlash != 0 && path.StoreType == kv.TiKV {
if ds.preferStoreType&h.PreferTiFlash != 0 && path.StoreType == kv.TiKV {
continue
}
if ds.preferStoreType&preferTiKV != 0 && path.StoreType == kv.TiFlash {
if ds.preferStoreType&h.PreferTiKV != 0 && path.StoreType == kv.TiFlash {
continue
}
var tblTask task
Expand Down Expand Up @@ -1290,7 +1291,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter
continue
}
// TiFlash storage do not support index scan.
if ds.preferStoreType&preferTiFlash != 0 {
if ds.preferStoreType&h.PreferTiFlash != 0 {
continue
}
idxTask, err := ds.convertToIndexScan(prop, candidate, opt)
Expand Down
2 changes: 1 addition & 1 deletion pkg/planner/core/indexmerge_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,7 +76,7 @@ func TestIndexMergePathGeneration(t *testing.T) {
err = Preprocess(context.Background(), sctx, stmt, WithPreprocessorReturn(&PreprocessorReturn{InfoSchema: is}))
require.NoError(t, err)
sctx := MockContext()
builder, _ := NewPlanBuilder().Init(sctx, is, &hint.BlockHintProcessor{})
builder, _ := NewPlanBuilder().Init(sctx, is, &hint.QBHintHandler{})
p, err := builder.Build(ctx, stmt)
if err != nil {
testdata.OnRecord(func() {
Expand Down
Loading

0 comments on commit ebd259c

Please sign in to comment.