logicalop

package
v1.1.0-beta.0...-ec288d9 Latest Latest
Warning

This package is not in the latest version of its module.

Go to latest
Published: Nov 29, 2024 License: Apache-2.0 Imports: 52 Imported by: 0

Documentation

Index

Constants

This section is empty.

Variables

This section is empty.

Functions

func AppendAddSelectionTraceStep

func AppendAddSelectionTraceStep(p base.LogicalPlan, child base.LogicalPlan, sel *LogicalSelection, opt *optimizetrace.LogicalOptimizeOp)

AppendAddSelectionTraceStep appends a trace step for adding a selection operator.

func AppendTableDualTraceStep

func AppendTableDualTraceStep(replaced base.LogicalPlan, dual base.LogicalPlan, conditions []expression.Expression, opt *optimizetrace.LogicalOptimizeOp)

AppendTableDualTraceStep appends a trace step for replacing a plan with a dual table.

func BuildLogicalJoinSchema

func BuildLogicalJoinSchema(joinType JoinType, join base.LogicalPlan) *expression.Schema

BuildLogicalJoinSchema builds the schema for join operator.

func CanPushToCopImpl

func CanPushToCopImpl(lp base.LogicalPlan, storeTp kv.StoreType, considerDual bool) bool

CanPushToCopImpl checks whether the logical plan can be pushed to coprocessor.

func Conds2TableDual

func Conds2TableDual(p base.LogicalPlan, conds []expression.Expression) base.LogicalPlan

Conds2TableDual builds a LogicalTableDual if cond is constant false or null.

func DeriveOtherConditions

func DeriveOtherConditions(
	p *LogicalJoin, leftSchema *expression.Schema, rightSchema *expression.Schema,
	deriveLeft bool, deriveRight bool) (
	leftCond []expression.Expression, rightCond []expression.Expression)

DeriveOtherConditions given a LogicalJoin, check the OtherConditions to see if we can derive more conditions for left/right child pushdown.

func HasMaxOneRow

func HasMaxOneRow(p base.LogicalPlan, childMaxOneRow []bool) bool

HasMaxOneRow returns if the LogicalPlan will output at most one row.

func IsSupportedSelectLockType

func IsSupportedSelectLockType(lockType ast.SelectLockType) bool

IsSupportedSelectLockType checks if the lockType is supported to acquire pessimsitic locks if necessary.

Types

type BaseLogicalPlan

type BaseLogicalPlan struct {
	baseimpl.Plan
	// contains filtered or unexported fields
}

BaseLogicalPlan is the common structure that used in logical plan.

func NewBaseLogicalPlan

func NewBaseLogicalPlan(ctx base.PlanContext, tp string, self base.LogicalPlan, qbOffset int) BaseLogicalPlan

NewBaseLogicalPlan is the basic constructor of BaseLogicalPlan.

func (*BaseLogicalPlan) BuildKeyInfo

func (p *BaseLogicalPlan) BuildKeyInfo(_ *expression.Schema, _ []*expression.Schema)

BuildKeyInfo implements LogicalPlan.<4th> interface.

func (*BaseLogicalPlan) BuildPlanTrace

func (p *BaseLogicalPlan) BuildPlanTrace() *tracing.PlanTrace

BuildPlanTrace implements Plan

func (*BaseLogicalPlan) CanPushToCop

func (p *BaseLogicalPlan) CanPushToCop(storeTp kv.StoreType) bool

CanPushToCop implements LogicalPlan.<21st> interface. it checks if it can be pushed to some stores. For TiKV, it only checks datasource. For TiFlash, it will check whether the operator is supported, but note that the check might be inaccurate.

func (*BaseLogicalPlan) ChildLen

func (p *BaseLogicalPlan) ChildLen() int

ChildLen returns the child length of BaseLogicalPlan.

func (*BaseLogicalPlan) Children

func (p *BaseLogicalPlan) Children() []base.LogicalPlan

Children implements LogicalPlan.<17th> interface.

func (*BaseLogicalPlan) ConstantPropagation

func (*BaseLogicalPlan) ConstantPropagation(_ base.LogicalPlan, _ int, _ *optimizetrace.LogicalOptimizeOp) (newRoot base.LogicalPlan)

ConstantPropagation implements the LogicalPlan.<8th> interface.

func (*BaseLogicalPlan) ConvertOuterToInnerJoin

func (p *BaseLogicalPlan) ConvertOuterToInnerJoin(predicates []expression.Expression) base.LogicalPlan

ConvertOuterToInnerJoin implements LogicalPlan.<24th> interface.

func (*BaseLogicalPlan) DeriveStats

func (p *BaseLogicalPlan) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements LogicalPlan.<11th> interface.

func (*BaseLogicalPlan) DeriveTopN

DeriveTopN implements the LogicalPlan.<6th> interface.

func (*BaseLogicalPlan) Equals

func (p *BaseLogicalPlan) Equals(other any) bool

Equals implements HashEquals.<1st> interface.

func (*BaseLogicalPlan) ExhaustPhysicalPlans

func (*BaseLogicalPlan) ExhaustPhysicalPlans(*property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements LogicalPlan.<14th> interface.

func (*BaseLogicalPlan) ExplainInfo

func (*BaseLogicalPlan) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*BaseLogicalPlan) ExtractColGroups

func (*BaseLogicalPlan) ExtractColGroups(_ [][]*expression.Column) [][]*expression.Column

ExtractColGroups implements LogicalPlan.<12th> interface.

func (*BaseLogicalPlan) ExtractCorrelatedCols

func (*BaseLogicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements LogicalPlan.<15th> interface.

func (*BaseLogicalPlan) ExtractFD

func (p *BaseLogicalPlan) ExtractFD() *fd.FDSet

ExtractFD implements LogicalPlan.<22nd> interface. It returns the children[0]'s fdSet if there are no adding/removing fd in this logic plan.

func (*BaseLogicalPlan) FDs

func (p *BaseLogicalPlan) FDs() *fd.FDSet

FDs returns the FDSet of BaseLogicalPlan.

func (*BaseLogicalPlan) FindBestTask

func (p *BaseLogicalPlan) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp,
	opt *optimizetrace.PhysicalOptimizeOp) (bestTask base.Task, cntPlan int64, err error)

FindBestTask implements LogicalPlan.<3rd> interface.

func (*BaseLogicalPlan) GetBaseLogicalPlan

func (p *BaseLogicalPlan) GetBaseLogicalPlan() base.LogicalPlan

GetBaseLogicalPlan implements LogicalPlan.<23rd> interface. It returns the baseLogicalPlan inside each logical plan.

func (*BaseLogicalPlan) GetLogicalTS4TaskMap

func (p *BaseLogicalPlan) GetLogicalTS4TaskMap() uint64

GetLogicalTS4TaskMap get the logical TimeStamp now to help rollback the TaskMap changes after that.

func (*BaseLogicalPlan) GetTask

GetTask returns the history recorded Task for specified property.

func (*BaseLogicalPlan) Hash64

func (p *BaseLogicalPlan) Hash64(h base2.Hasher)

Hash64 implements HashEquals.<0th> interface.

func (*BaseLogicalPlan) HashCode

func (p *BaseLogicalPlan) HashCode() []byte

HashCode implements LogicalPlan.<0th> interface.

func (*BaseLogicalPlan) MaxOneRow

func (p *BaseLogicalPlan) MaxOneRow() bool

MaxOneRow implements the LogicalPlan.<16th> interface.

func (*BaseLogicalPlan) OutputNames

func (p *BaseLogicalPlan) OutputNames() types.NameSlice

OutputNames implements Plan Schema interface.

func (*BaseLogicalPlan) PredicatePushDown

PredicatePushDown implements LogicalPlan.<1st> interface.

func (*BaseLogicalPlan) PredicateSimplification

func (p *BaseLogicalPlan) PredicateSimplification(opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PredicateSimplification implements the LogicalPlan.<7th> interface.

func (*BaseLogicalPlan) PreparePossibleProperties

func (*BaseLogicalPlan) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements LogicalPlan.<13th> interface.

func (*BaseLogicalPlan) PruneColumns

func (p *BaseLogicalPlan) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements LogicalPlan.<2nd> interface.

func (*BaseLogicalPlan) PullUpConstantPredicates

func (*BaseLogicalPlan) PullUpConstantPredicates() []expression.Expression

PullUpConstantPredicates implements the LogicalPlan.<9th> interface.

func (*BaseLogicalPlan) PushDownTopN

func (p *BaseLogicalPlan) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PushDownTopN implements the LogicalPlan.<5th> interface.

func (*BaseLogicalPlan) RecursiveDeriveStats

func (p *BaseLogicalPlan) RecursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error)

RecursiveDeriveStats implements LogicalPlan.<10th> interface.

func (*BaseLogicalPlan) RollBackTaskMap

func (p *BaseLogicalPlan) RollBackTaskMap(ts uint64)

RollBackTaskMap implements LogicalPlan.<20th> interface.

func (*BaseLogicalPlan) Schema

func (p *BaseLogicalPlan) Schema() *expression.Schema

Schema implements Plan Schema interface.

func (*BaseLogicalPlan) Self

func (p *BaseLogicalPlan) Self() base.LogicalPlan

Self returns the self LogicalPlan of BaseLogicalPlan.

func (*BaseLogicalPlan) SetChild

func (p *BaseLogicalPlan) SetChild(i int, child base.LogicalPlan)

SetChild implements LogicalPlan.<19th> interface.

func (*BaseLogicalPlan) SetChildren

func (p *BaseLogicalPlan) SetChildren(children ...base.LogicalPlan)

SetChildren implements LogicalPlan.<18th> interface.

func (*BaseLogicalPlan) SetFDs

func (p *BaseLogicalPlan) SetFDs(fd *fd.FDSet)

SetFDs sets the FDSet of BaseLogicalPlan.

func (*BaseLogicalPlan) SetMaxOneRow

func (p *BaseLogicalPlan) SetMaxOneRow(b bool)

SetMaxOneRow sets the maxOneRow of BaseLogicalPlan.

func (*BaseLogicalPlan) SetOutputNames

func (p *BaseLogicalPlan) SetOutputNames(names types.NameSlice)

SetOutputNames implements Plan Schema interface.

func (*BaseLogicalPlan) SetSelf

func (p *BaseLogicalPlan) SetSelf(s base.LogicalPlan)

SetSelf sets the self LogicalPlan of BaseLogicalPlan.

func (*BaseLogicalPlan) StoreTask

func (p *BaseLogicalPlan) StoreTask(prop *property.PhysicalProperty, task base.Task)

StoreTask records Task for specified property as <k,v>.

type CTEClass

type CTEClass struct {
	// The union between seed part and recursive part is DISTINCT or DISTINCT ALL.
	IsDistinct bool
	// SeedPartLogicalPlan and RecursivePartLogicalPlan are the logical plans for the seed part and recursive part of this CTE.
	SeedPartLogicalPlan base.LogicalPlan
	// RecursivePartLogicalPlan is nil if this CTE is not a recursive CTE.
	RecursivePartLogicalPlan base.LogicalPlan
	// SeedPartPhysicalPlan and RecursivePartPhysicalPlan are the physical plans for the seed part and recursive part of this CTE.
	SeedPartPhysicalPlan      base.PhysicalPlan
	RecursivePartPhysicalPlan base.PhysicalPlan
	// storageID for this CTE.
	IDForStorage int
	// OptFlag is the OptFlag for the whole CTE.
	OptFlag   uint64
	HasLimit  bool
	LimitBeg  uint64
	LimitEnd  uint64
	IsInApply bool
	// PushDownPredicates may be push-downed by different references.
	PushDownPredicates []expression.Expression
	ColumnMap          map[string]*expression.Column
	IsOuterMostCTE     bool
}

CTEClass holds the information and plan for a CTE. Most of the fields in this struct are the same as cteInfo. But the cteInfo is used when building the plan, and CTEClass is used also for building the executor.

func (*CTEClass) MemoryUsage

func (cc *CTEClass) MemoryUsage() (sum int64)

MemoryUsage return the memory usage of CTEClass

type DataSource

type DataSource struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	AstIndexHints []*ast.IndexHint
	IndexHints    []h.HintedIndex
	Table         table.Table
	TableInfo     *model.TableInfo `hash64-equals:"true"`
	Columns       []*model.ColumnInfo
	DBName        pmodel.CIStr

	TableAsName *pmodel.CIStr `hash64-equals:"true"`
	// IndexMergeHints are the hint for indexmerge.
	IndexMergeHints []h.HintedIndex
	// PushedDownConds are the conditions that will be pushed down to coprocessor.
	PushedDownConds []expression.Expression `hash64-equals:"true"`
	// AllConds contains all the filters on this table. For now it's maintained
	// in predicate push down and used in partition pruning/index merge.
	AllConds []expression.Expression `hash64-equals:"true"`

	StatisticTable *statistics.Table
	TableStats     *property.StatsInfo

	// PossibleAccessPaths stores all the possible access path for physical plan, including table scan.
	PossibleAccessPaths []*util.AccessPath

	// The data source may be a partition, rather than a real table.
	PartitionDefIdx *int
	PhysicalTableID int64
	PartitionNames  []pmodel.CIStr

	// handleCol represents the handle column for the datasource, either the
	// int primary key column or extra handle column.
	// handleCol *expression.Column
	HandleCols          util.HandleCols
	UnMutableHandleCols util.HandleCols
	// TblCols contains the original columns of table before being pruned, and it
	// is used for estimating table scan cost.
	TblCols []*expression.Column
	// CommonHandleCols and CommonHandleLens save the info of primary key which is the clustered index.
	CommonHandleCols []*expression.Column
	CommonHandleLens []int
	// TblColHists contains the Histogram of all original table columns,
	// it is converted from StatisticTable, and used for IO/network cost estimating.
	TblColHists *statistics.HistColl
	// PreferStoreType means the DataSource is enforced to which storage.
	PreferStoreType int `hash64-equals:"true"`
	// PreferPartitions store the map, the key represents store type, the value represents the partition name list.
	PreferPartitions map[int][]pmodel.CIStr
	SampleInfo       *tablesampler.TableSampleInfo
	IS               infoschema.InfoSchema
	// IsForUpdateRead should be true in either of the following situations
	// 1. use `inside insert`, `update`, `delete` or `select for update` statement
	// 2. isolation level is RC
	IsForUpdateRead bool `hash64-equals:"true"`

	// contain unique index and the first field is tidb_shard(),
	// such as (tidb_shard(a), a ...), the fields are more than 2
	ContainExprPrefixUk bool

	// ColsRequiringFullLen is the columns that must be fetched with full length.
	// It is used to decide whether single scan is enough when reading from an index.
	ColsRequiringFullLen []*expression.Column

	// AccessPathMinSelectivity is the minimal selectivity among the access paths.
	// It's calculated after we generated the access paths and estimated row count for them, and before entering findBestTask.
	// It considers CountAfterIndex for index paths and CountAfterAccess for table paths and index merge paths.
	AccessPathMinSelectivity float64
}

DataSource represents a tableScan without condition push down.

func (*DataSource) BuildKeyInfo

func (ds *DataSource) BuildKeyInfo(selfSchema *expression.Schema, _ []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*DataSource) Convert2Gathers

func (ds *DataSource) Convert2Gathers() (gathers []base.LogicalPlan)

Convert2Gathers builds logical TiKVSingleGathers from DataSource.

func (*DataSource) DeriveStats

func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements base.LogicalPlan.<11th> interface.

func (*DataSource) Equals

func (op *DataSource) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *DataSource pointer.

func (*DataSource) ExplainInfo

func (ds *DataSource) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*DataSource) ExtractCorrelatedCols

func (ds *DataSource) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.

func (*DataSource) ExtractFD

func (ds *DataSource) ExtractFD() *fd.FDSet

ExtractFD implements the base.LogicalPlan.<22nd> interface.

func (*DataSource) FindBestTask

func (ds *DataSource) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp,
	opt *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error)

FindBestTask implements the base.LogicalPlan.<3rd> interface. It will enumerate all the available indices and choose a plan with least cost.

func (*DataSource) GetPKIsHandleCol

func (ds *DataSource) GetPKIsHandleCol() *expression.Column

GetPKIsHandleCol gets the handle column if the PKIsHandle is true, otherwise, returns the ExtraHandleColumn.

func (*DataSource) Hash64

func (op *DataSource) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (DataSource) Init

func (ds DataSource) Init(ctx base.PlanContext, offset int) *DataSource

Init initializes DataSource.

func (*DataSource) NewExtraHandleSchemaCol

func (ds *DataSource) NewExtraHandleSchemaCol() *expression.Column

NewExtraHandleSchemaCol creates a new column for extra handle.

func (*DataSource) PredicatePushDown

func (ds *DataSource) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*DataSource) PredicateSimplification

func (ds *DataSource) PredicateSimplification(*optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PredicateSimplification implements the base.LogicalPlan.<7th> interface.

func (*DataSource) PreparePossibleProperties

func (ds *DataSource) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

func (*DataSource) PruneColumns

func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

type FrameBound

type FrameBound struct {
	Type      ast.BoundType
	UnBounded bool
	Num       uint64
	// CalcFuncs is used for range framed windows.
	// We will build the date_add or date_sub functions for frames like `INTERVAL '2:30' MINUTE_SECOND FOLLOWING`,
	// and plus or minus for frames like `1 preceding`.
	CalcFuncs []expression.Expression
	// Sometimes we need to cast order by column to a specific type when frame type is range
	CompareCols []expression.Expression
	// CmpFuncs is used to decide whether one row is included in the current frame.
	CmpFuncs []expression.CompareFunc
	// This field is used for passing information to tiflash
	CmpDataType tipb.RangeCmpDataType
	// IsExplicitRange marks if this range explicitly appears in the sql
	IsExplicitRange bool
}

FrameBound is the boundary of a frame.

func (*FrameBound) Clone

func (fb *FrameBound) Clone() *FrameBound

Clone copies a frame bound totally.

func (*FrameBound) Equals

func (fb *FrameBound) Equals(other any) bool

Equals implement HashEquals interface.

func (*FrameBound) Hash64

func (fb *FrameBound) Hash64(h base2.Hasher)

Hash64 implement HashEquals interface.

func (*FrameBound) ToPB

func (fb *FrameBound) ToPB(ctx *base.BuildPBContext) (*tipb.WindowFrameBound, error)

ToPB converts FrameBound to tipb structure.

func (*FrameBound) UpdateCmpFuncsAndCmpDataType

func (fb *FrameBound) UpdateCmpFuncsAndCmpDataType(cmpDataType types.EvalType)

UpdateCmpFuncsAndCmpDataType updates CmpFuncs and CmpDataType.

func (*FrameBound) UpdateCompareCols

func (fb *FrameBound) UpdateCompareCols(ctx sessionctx.Context, orderByCols []*expression.Column) error

UpdateCompareCols will update CompareCols.

type JoinType

type JoinType int

JoinType contains CrossJoin, InnerJoin, LeftOuterJoin, RightOuterJoin, SemiJoin, AntiJoin.

const (
	// InnerJoin means inner join.
	InnerJoin JoinType = iota
	// LeftOuterJoin means left join.
	LeftOuterJoin
	// RightOuterJoin means right join.
	RightOuterJoin
	// SemiJoin means if row a in table A matches some rows in B, just output a.
	SemiJoin
	// AntiSemiJoin means if row a in table A does not match any row in B, then output a.
	AntiSemiJoin
	// LeftOuterSemiJoin means if row a in table A matches some rows in B, output (a, true), otherwise, output (a, false).
	LeftOuterSemiJoin
	// AntiLeftOuterSemiJoin means if row a in table A matches some rows in B, output (a, false), otherwise, output (a, true).
	AntiLeftOuterSemiJoin
)

func (JoinType) IsOuterJoin

func (tp JoinType) IsOuterJoin() bool

IsOuterJoin returns if this joiner is an outer joiner

func (JoinType) IsSemiJoin

func (tp JoinType) IsSemiJoin() bool

IsSemiJoin returns if this joiner is a semi/anti-semi joiner

func (JoinType) String

func (tp JoinType) String() string

type LogicalAggregation

type LogicalAggregation struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	AggFuncs     []*aggregation.AggFuncDesc `hash64-equals:"true"`
	GroupByItems []expression.Expression    `hash64-equals:"true"`

	// PreferAggType And PreferAggToCop stores aggregation hint information.
	PreferAggType  uint
	PreferAggToCop bool

	PossibleProperties [][]*expression.Column `hash64-equals:"true"`
	InputCount         float64                // InputCount is the input count of this plan.

	// NoCopPushDown indicates if planner must not push this agg down to coprocessor.
	// It is true when the agg is in the outer child tree of apply.
	NoCopPushDown bool
}

LogicalAggregation represents an aggregate plan.

func (*LogicalAggregation) BuildKeyInfo

func (la *LogicalAggregation) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*LogicalAggregation) BuildSelfKeyInfo

func (la *LogicalAggregation) BuildSelfKeyInfo(selfSchema *expression.Schema)

BuildSelfKeyInfo builds the key information for the aggregation itself.

func (*LogicalAggregation) CanPullUp

func (la *LogicalAggregation) CanPullUp() bool

CanPullUp checks if an aggregation can be pulled up. An aggregate function like count(*) cannot be pulled up.

func (*LogicalAggregation) CanPushToCop

func (la *LogicalAggregation) CanPushToCop(storeTp kv.StoreType) bool

CanPushToCop implements base.LogicalPlan.<21st> interface.

func (*LogicalAggregation) CopyAggHints

func (la *LogicalAggregation) CopyAggHints(agg *LogicalAggregation)

CopyAggHints copies the aggHints from another LogicalAggregation.

func (*LogicalAggregation) DeriveStats

func (la *LogicalAggregation) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, childSchema []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implement base.LogicalPlan.<11th> interface.

func (*LogicalAggregation) DistinctArgsMeetsProperty

func (la *LogicalAggregation) DistinctArgsMeetsProperty() bool

DistinctArgsMeetsProperty checks if the distinct args meet the property.

func (*LogicalAggregation) Equals

func (op *LogicalAggregation) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalAggregation pointer.

func (*LogicalAggregation) ExhaustPhysicalPlans

func (la *LogicalAggregation) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalAggregation) ExplainInfo

func (la *LogicalAggregation) ExplainInfo() string

ExplainInfo implements base.Plan.<4th> interface.

func (*LogicalAggregation) ExtractColGroups

func (la *LogicalAggregation) ExtractColGroups(_ [][]*expression.Column) [][]*expression.Column

ExtractColGroups implements base.LogicalPlan.<12th> interface.

func (*LogicalAggregation) ExtractCorrelatedCols

func (la *LogicalAggregation) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.

func (*LogicalAggregation) ExtractFD

func (la *LogicalAggregation) ExtractFD() *fd.FDSet

ExtractFD implements base.LogicalPlan.<22nd> interface. 1: In most of the cases, using FDs to check the only_full_group_by problem should be done in the buildAggregation phase by extracting the bottom-up FDs graph from the `p` --- the sub plan tree that has already been built.

2: and this requires that some conditions push-down into the `p` like selection should be done before building aggregation, otherwise, 'a=1 and a can occur in the select lists of a group by' will be miss-checked because it doesn't be implied in the known FDs graph.

3: when a logical agg is built, it's schema columns indicates what the permitted-non-agg columns is. Therefore, we shouldn't depend on logicalAgg.ExtractFD() to finish the only_full_group_by checking problem rather than by 1 & 2.

func (*LogicalAggregation) GetGroupByCols

func (la *LogicalAggregation) GetGroupByCols() []*expression.Column

GetGroupByCols returns the columns that are group-by items. For example, `group by a, b, c+d` will return [a, b].

func (*LogicalAggregation) GetPotentialPartitionKeys

func (la *LogicalAggregation) GetPotentialPartitionKeys() []*property.MPPPartitionColumn

GetPotentialPartitionKeys return potential partition keys for aggregation, the potential partition keys are the group by keys

func (*LogicalAggregation) GetUsedCols

func (la *LogicalAggregation) GetUsedCols() (usedCols []*expression.Column)

GetUsedCols extracts all of the Columns used by agg including GroupByItems and AggFuncs.

func (*LogicalAggregation) HasDistinct

func (la *LogicalAggregation) HasDistinct() bool

HasDistinct shows whether LogicalAggregation has functions with distinct.

func (*LogicalAggregation) HasOrderBy

func (la *LogicalAggregation) HasOrderBy() bool

HasOrderBy shows whether LogicalAggregation has functions with order-by items.

func (*LogicalAggregation) Hash64

func (op *LogicalAggregation) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalAggregation) Init

Init initializes LogicalAggregation.

func (*LogicalAggregation) IsCompleteModeAgg

func (la *LogicalAggregation) IsCompleteModeAgg() bool

IsCompleteModeAgg returns if all of the AggFuncs are CompleteMode.

func (*LogicalAggregation) IsPartialModeAgg

func (la *LogicalAggregation) IsPartialModeAgg() bool

IsPartialModeAgg returns if all of the AggFuncs are partialMode.

func (*LogicalAggregation) PredicatePushDown

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalAggregation) PreparePossibleProperties

func (la *LogicalAggregation) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

func (*LogicalAggregation) PruneColumns

func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

func (*LogicalAggregation) ReplaceExprColumns

func (la *LogicalAggregation) ReplaceExprColumns(replace map[string]*expression.Column)

ReplaceExprColumns implements base.Plan.<5th> interface.

func (*LogicalAggregation) ResetHintIfConflicted

func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferStream bool)

ResetHintIfConflicted resets the PreferAggType if they are conflicted, and returns the two PreferAggType hints.

type LogicalApply

type LogicalApply struct {
	LogicalJoin `hash64-equals:"true"`

	CorCols []*expression.CorrelatedColumn `hash64-equals:"true"`
	// NoDecorrelate is from /*+ no_decorrelate() */ hint.
	NoDecorrelate bool `hash64-equals:"true"`
}

LogicalApply gets one row from outer executor and gets one row from inner executor according to outer row.

func (*LogicalApply) CanPullUpAgg

func (la *LogicalApply) CanPullUpAgg() bool

CanPullUpAgg checks if an apply can pull an aggregation up.

func (*LogicalApply) DeCorColFromEqExpr

func (la *LogicalApply) DeCorColFromEqExpr(expr expression.Expression) expression.Expression

DeCorColFromEqExpr checks whether it's an equal condition of form `col = correlated col`. If so we will change the decorrelated column to normal column to make a new equal condition.

func (*LogicalApply) DeriveStats

func (la *LogicalApply) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, childSchema []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements base.LogicalPlan.<11th> interface.

func (*LogicalApply) Equals

func (op *LogicalApply) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalApply pointer.

func (*LogicalApply) ExhaustPhysicalPlans

func (la *LogicalApply) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalApply) ExplainInfo

func (la *LogicalApply) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalApply) ExtractColGroups

func (la *LogicalApply) ExtractColGroups(colGroups [][]*expression.Column) [][]*expression.Column

ExtractColGroups implements base.LogicalPlan.<12th> interface.

func (*LogicalApply) ExtractCorrelatedCols

func (la *LogicalApply) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.

func (*LogicalApply) ExtractFD

func (la *LogicalApply) ExtractFD() *fd.FDSet

ExtractFD implements the base.LogicalPlan.<22nd> interface.

func (*LogicalApply) Hash64

func (op *LogicalApply) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalApply) Init

func (la LogicalApply) Init(ctx base.PlanContext, offset int) *LogicalApply

Init initializes LogicalApply.

func (*LogicalApply) PruneColumns

func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

func (*LogicalApply) ReplaceExprColumns

func (la *LogicalApply) ReplaceExprColumns(replace map[string]*expression.Column)

ReplaceExprColumns implements base.LogicalPlan interface.

type LogicalCTE

type LogicalCTE struct {
	LogicalSchemaProducer

	Cte       *CTEClass
	CteAsName model.CIStr
	CteName   model.CIStr
	SeedStat  *property.StatsInfo

	OnlyUsedAsStorage bool
}

LogicalCTE is for CTE.

func (*LogicalCTE) DeriveStats

func (p *LogicalCTE) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements the base.LogicalPlan.<11th> interface.

func (*LogicalCTE) ExhaustPhysicalPlans

func (p *LogicalCTE) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements the base.LogicalPlan.<14th> interface.

func (*LogicalCTE) ExtractCorrelatedCols

func (p *LogicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements the base.LogicalPlan.<15th> interface.

func (*LogicalCTE) FindBestTask

func (p *LogicalCTE) FindBestTask(prop *property.PhysicalProperty, counter *base.PlanCounterTp, pop *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error)

FindBestTask implements the base.LogicalPlan.<3rd> interface.

func (LogicalCTE) Init

func (p LogicalCTE) Init(ctx base.PlanContext, offset int) *LogicalCTE

Init only assigns type and context.

func (*LogicalCTE) PredicatePushDown

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalCTE) PruneColumns

PruneColumns implements the base.LogicalPlan.<2nd> interface. LogicalCTE just do an empty function call. It's logical optimize is indivisual phase.

func (*LogicalCTE) PushDownTopN

func (p *LogicalCTE) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PushDownTopN implements the base.LogicalPlan.<5th> interface.

type LogicalCTETable

type LogicalCTETable struct {
	LogicalSchemaProducer

	SeedStat     *property.StatsInfo
	Name         string
	IDForStorage int

	// SeedSchema is only used in columnStatsUsageCollector to get column mapping
	SeedSchema *expression.Schema
}

LogicalCTETable is for CTE table

func (*LogicalCTETable) DeriveStats

DeriveStats implements the base.LogicalPlan.<11th> interface.

func (*LogicalCTETable) FindBestTask

FindBestTask implements the base.LogicalPlan.<3rd> interface.

func (LogicalCTETable) Init

func (p LogicalCTETable) Init(ctx base.PlanContext, offset int) *LogicalCTETable

Init only assigns type and context.

type LogicalExpand

type LogicalExpand struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	// distinct group by columns. (maybe projected below if it's a non-col)
	DistinctGroupByCol  []*expression.Column `hash64-equals:"true"`
	DistinctGbyColNames []*types.FieldName
	// keep the old gbyExprs for resolve cases like grouping(a+b), the args:
	// a+b should be resolved to new projected gby col according to ref pos.
	DistinctGbyExprs []expression.Expression `hash64-equals:"true"`

	// rollup grouping sets.
	DistinctSize       int                     `hash64-equals:"true"`
	RollupGroupingSets expression.GroupingSets `hash64-equals:"true"`
	RollupID2GIDS      map[int]map[uint64]struct{}
	RollupGroupingIDs  []uint64

	// The level projections is generated from grouping sets,make execution more clearly.
	LevelExprs [][]expression.Expression `hash64-equals:"true"`

	// The generated column names. Eg: "grouping_id" and so on.
	ExtraGroupingColNames []string

	// GroupingMode records the grouping id allocation mode.
	GroupingMode tipb.GroupingMode

	// The GID and GPos column generated by logical expand if any.
	GID      *expression.Column `hash64-equals:"true"`
	GIDName  *types.FieldName
	GPos     *expression.Column `hash64-equals:"true"`
	GPosName *types.FieldName
}

LogicalExpand represents a logical Expand OP serves for data replication requirement.

func (*LogicalExpand) Equals

func (op *LogicalExpand) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalExpand pointer.

func (*LogicalExpand) ExhaustPhysicalPlans

func (p *LogicalExpand) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalExpand) ExtractCorrelatedCols

func (p *LogicalExpand) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.

func (*LogicalExpand) ExtractFD

func (p *LogicalExpand) ExtractFD() *fd.FDSet

ExtractFD implements the base.LogicalPlan.<22nd> interface, extracting the FD from bottom up.

func (*LogicalExpand) GenLevelProjections

func (p *LogicalExpand) GenLevelProjections()

GenLevelProjections is used to generate level projections after all the necessary logical optimization is done such as column pruning.

func (*LogicalExpand) GenerateGroupingIDIncrementModeNumericSet

func (p *LogicalExpand) GenerateGroupingIDIncrementModeNumericSet(oneSetOffset int) uint64

GenerateGroupingIDIncrementModeNumericSet is used to generate grouping ids when the num of grouping sets is greater than 64. Under this circumstance, bitAnd uint64 doesn't have enough capacity to set those bits, so incremental grouping ID set is chosen.

func (*LogicalExpand) GenerateGroupingIDModeBitAnd

func (p *LogicalExpand) GenerateGroupingIDModeBitAnd(oneSet expression.GroupingSet) uint64

GenerateGroupingIDModeBitAnd is used to generate convenient groupingID for quick computation of grouping function. A bit in the bitmask is corresponding to an attribute in the group by attributes sequence, the selected attribute has corresponding bit set to 0 and otherwise set to 1. Example, if we have GroupBy attributes(a,b,c,d), the bitmask 5 (whose binary form is 0101) represents grouping set (a,c).

func (*LogicalExpand) GenerateGroupingMarks

func (p *LogicalExpand) GenerateGroupingMarks(sourceCols []*expression.Column) []map[uint64]struct{}

GenerateGroupingMarks generate the groupingMark for the source column specified in grouping function.

func (*LogicalExpand) GetUsedCols

func (*LogicalExpand) GetUsedCols() (usedCols []*expression.Column)

GetUsedCols extracts all of the Columns used by proj.

func (*LogicalExpand) Hash64

func (op *LogicalExpand) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalExpand) Init

func (p LogicalExpand) Init(ctx base.PlanContext, offset int) *LogicalExpand

Init initializes LogicalProjection.

func (*LogicalExpand) PredicatePushDown

func (p *LogicalExpand) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan)

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalExpand) PruneColumns

func (p *LogicalExpand) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implement the base.LogicalPlan.<2nd> interface. logicExpand is built in the logical plan building phase, where all the column prune is not done yet. So the expand projection expressions is meaningless if it built at that time. (we only maintain its schema, while the level projection expressions construction is left to the last logical optimize rule)

so when do the rule_column_pruning here, we just prune the schema is enough.

func (*LogicalExpand) ResolveGroupingFuncArgsInGroupBy

func (p *LogicalExpand) ResolveGroupingFuncArgsInGroupBy(groupingFuncArgs []expression.Expression) ([]*expression.Column, error)

ResolveGroupingFuncArgsInGroupBy checks whether grouping function args is in grouping items.

func (*LogicalExpand) TrySubstituteExprWithGroupingSetCol

func (p *LogicalExpand) TrySubstituteExprWithGroupingSetCol(expr expression.Expression) (expression.Expression, bool)

TrySubstituteExprWithGroupingSetCol is used to substitute the original gby expression with new gby col.

type LogicalIndexScan

type LogicalIndexScan struct {
	LogicalSchemaProducer
	// DataSource should be read-only here.
	Source       *DataSource
	IsDoubleRead bool

	EqCondCount int
	AccessConds expression.CNFExprs
	Ranges      []*ranger.Range

	Index          *model.IndexInfo
	Columns        []*model.ColumnInfo
	FullIdxCols    []*expression.Column
	FullIdxColLens []int
	IdxCols        []*expression.Column
	IdxColLens     []int
}

LogicalIndexScan is the logical index scan operator for TiKV.

func (*LogicalIndexScan) BuildKeyInfo

func (is *LogicalIndexScan) BuildKeyInfo(selfSchema *expression.Schema, _ []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*LogicalIndexScan) DeriveStats

func (is *LogicalIndexScan) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements base.LogicalPlan.<11th> interface.

func (*LogicalIndexScan) ExplainInfo

func (is *LogicalIndexScan) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalIndexScan) GetPKIsHandleCol

func (is *LogicalIndexScan) GetPKIsHandleCol(schema *expression.Schema) *expression.Column

GetPKIsHandleCol gets the handle column if PKIsHandle.

func (LogicalIndexScan) Init

func (is LogicalIndexScan) Init(ctx base.PlanContext, offset int) *LogicalIndexScan

Init initializes LogicalIndexScan.

func (*LogicalIndexScan) MatchIndexProp

func (is *LogicalIndexScan) MatchIndexProp(prop *property.PhysicalProperty) (match bool)

MatchIndexProp checks if the indexScan can match the required property.

func (*LogicalIndexScan) PreparePossibleProperties

func (is *LogicalIndexScan) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

type LogicalJoin

type LogicalJoin struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	JoinType      JoinType `hash64-equals:"true"`
	Reordered     bool
	CartesianJoin bool
	StraightJoin  bool

	// HintInfo stores the join algorithm hint information specified by client.
	HintInfo            *utilhint.PlanHints
	PreferJoinType      uint
	PreferJoinOrder     bool
	LeftPreferJoinType  uint
	RightPreferJoinType uint

	EqualConditions []*expression.ScalarFunction `hash64-equals:"true"`
	// NAEQConditions means null aware equal conditions, which is used for null aware semi joins.
	NAEQConditions  []*expression.ScalarFunction `hash64-equals:"true"`
	LeftConditions  expression.CNFExprs          `hash64-equals:"true"`
	RightConditions expression.CNFExprs          `hash64-equals:"true"`
	OtherConditions expression.CNFExprs          `hash64-equals:"true"`

	LeftProperties  [][]*expression.Column
	RightProperties [][]*expression.Column

	// DefaultValues is only used for left/right outer join, which is values the inner row's should be when the outer table
	// doesn't match any inner table's row.
	// That it's nil just means the default values is a slice of NULL.
	// Currently, only `aggregation push down` phase will set this.
	DefaultValues []types.Datum

	// FullSchema contains all the columns that the Join can output. It's ordered as [outer schema..., inner schema...].
	// This is useful for natural joins and "using" joins. In these cases, the join key columns from the
	// inner side (or the right side when it's an inner join) will not be in the schema of Join.
	// But upper operators should be able to find those "redundant" columns, and the user also can specifically select
	// those columns, so we put the "redundant" columns here to make them be able to be found.
	//
	// For example:
	// create table t1(a int, b int); create table t2(a int, b int);
	// select * from t1 join t2 using (b);
	// schema of the Join will be [t1.b, t1.a, t2.a]; FullSchema will be [t1.a, t1.b, t2.a, t2.b].
	//
	// We record all columns and keep them ordered is for correctly handling SQLs like
	// select t1.*, t2.* from t1 join t2 using (b);
	// (*PlanBuilder).unfoldWildStar() handles the schema for such case.
	FullSchema *expression.Schema
	FullNames  types.NameSlice

	// EqualCondOutCnt indicates the estimated count of joined rows after evaluating `EqualConditions`.
	EqualCondOutCnt float64
}

LogicalJoin is the logical join plan.

func (*LogicalJoin) AppendJoinConds

func (p *LogicalJoin) AppendJoinConds(eq []*expression.ScalarFunction, left, right, other []expression.Expression)

AppendJoinConds appends new join conditions.

func (*LogicalJoin) AttachOnConds

func (p *LogicalJoin) AttachOnConds(onConds []expression.Expression)

AttachOnConds extracts on conditions for join and set the `EqualConditions`, `LeftConditions`, `RightConditions` and `OtherConditions` by the result of extract.

func (*LogicalJoin) BuildKeyInfo

func (p *LogicalJoin) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements the base.LogicalPlan.<4th> interface.

func (*LogicalJoin) ColumnSubstituteAll

func (p *LogicalJoin) ColumnSubstituteAll(schema *expression.Schema, exprs []expression.Expression) (hasFail bool)

ColumnSubstituteAll is used in projection elimination in apply de-correlation. Substitutions for all conditions should be successful, otherwise, we should keep all conditions unchanged.

func (*LogicalJoin) ConstantPropagation

func (p *LogicalJoin) ConstantPropagation(parentPlan base.LogicalPlan, currentChildIdx int, opt *optimizetrace.LogicalOptimizeOp) (newRoot base.LogicalPlan)

ConstantPropagation implements the base.LogicalPlan.<8th> interface. about the logic of constant propagation in From List. Query: select * from t, (select a, b from s where s.a>1) tmp where tmp.a=t.a Origin logical plan:

      +----------------+
      |  LogicalJoin   |
      +-------^--------+
              |
+-------------+--------------+
|                            |

+-----+------+ +------+------+ | Projection | | TableScan | +-----^------+ +-------------+

|
|

+-----+------+ | Selection | | s.a>1 | +------------+

  1. 'PullUpConstantPredicates': Call this function until find selection and pull up the constant predicate layer by layer LogicalSelection: find the s.a>1 LogicalProjection: get the s.a>1 and pull up it, changed to tmp.a>1
  2. 'addCandidateSelection': Add selection above of LogicalJoin, put all predicates pulled up from the lower layer into the current new selection. LogicalSelection: tmp.a >1

Optimized plan:

      +----------------+
      |  Selection     |
      |    tmp.a>1     |
      +-------^--------+
              |
      +-------+--------+
      |  LogicalJoin   |
      +-------^--------+
              |
+-------------+--------------+
|                            |

+-----+------+ +------+------+ | Projection | | TableScan | +-----^------+ +-------------+

|
|

+-----+------+ | Selection | | s.a>1 | +------------+

Return nil if the root of plan has not been changed Return new root if the root of plan is changed to selection

func (*LogicalJoin) ConvertOuterToInnerJoin

func (p *LogicalJoin) ConvertOuterToInnerJoin(predicates []expression.Expression) base.LogicalPlan

ConvertOuterToInnerJoin implements base.LogicalPlan.<24th> interface.

func (*LogicalJoin) Decorrelate

func (p *LogicalJoin) Decorrelate(schema *expression.Schema)

Decorrelate eliminate the correlated column with if the col is in schema.

func (*LogicalJoin) DeriveStats

func (p *LogicalJoin) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, childSchema []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements the base.LogicalPlan.<11th> interface. If the type of join is SemiJoin, the selectivity of it will be same as selection's. If the type of join is LeftOuterSemiJoin, it will not add or remove any row. The last column is a boolean value, whose NDV should be two. If the type of join is inner/outer join, the output of join(s, t) should be N(s) * N(t) / (V(s.key) * V(t.key)) * Min(s.key, t.key). N(s) stands for the number of rows in relation s. V(s.key) means the NDV of join key in s. This is a quite simple strategy: We assume every bucket of relation which will participate join has the same number of rows, and apply cross join for every matched bucket.

func (*LogicalJoin) Equals

func (op *LogicalJoin) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalJoin pointer.

func (*LogicalJoin) ExhaustPhysicalPlans

func (p *LogicalJoin) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements the base.LogicalPlan.<14th> interface. it can generates hash join, index join and sort merge join. Firstly we check the hint, if hint is figured by user, we force to choose the corresponding physical plan. If the hint is not matched, it will get other candidates. If the hint is not figured, we will pick all candidates.

func (*LogicalJoin) ExplainInfo

func (p *LogicalJoin) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalJoin) ExtractColGroups

func (p *LogicalJoin) ExtractColGroups(colGroups [][]*expression.Column) [][]*expression.Column

ExtractColGroups implements the base.LogicalPlan.<12th> interface.

func (*LogicalJoin) ExtractCorrelatedCols

func (p *LogicalJoin) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements the base.LogicalPlan.<15th> interface.

func (*LogicalJoin) ExtractFD

func (p *LogicalJoin) ExtractFD() *funcdep.FDSet

ExtractFD implements the base.LogicalPlan.<22th> interface.

func (*LogicalJoin) ExtractFDForInnerJoin

func (p *LogicalJoin) ExtractFDForInnerJoin(filtersFromApply []expression.Expression) *funcdep.FDSet

ExtractFDForInnerJoin extracts FD for inner join.

func (*LogicalJoin) ExtractFDForOuterJoin

func (p *LogicalJoin) ExtractFDForOuterJoin(filtersFromApply []expression.Expression) *funcdep.FDSet

ExtractFDForOuterJoin extracts FD for outer join.

func (*LogicalJoin) ExtractFDForSemiJoin

func (p *LogicalJoin) ExtractFDForSemiJoin(filtersFromApply []expression.Expression) *funcdep.FDSet

ExtractFDForSemiJoin extracts FD for semi join.

func (*LogicalJoin) ExtractJoinKeys

func (p *LogicalJoin) ExtractJoinKeys(childIdx int) *expression.Schema

ExtractJoinKeys extract join keys as a schema for child with childIdx.

func (*LogicalJoin) ExtractOnCondition

func (p *LogicalJoin) ExtractOnCondition(
	conditions []expression.Expression,
	leftSchema *expression.Schema,
	rightSchema *expression.Schema,
	deriveLeft bool,
	deriveRight bool) (eqCond []*expression.ScalarFunction, leftCond []expression.Expression,
	rightCond []expression.Expression, otherCond []expression.Expression)

ExtractOnCondition divide conditions in CNF of join node into 4 groups. These conditions can be where conditions, join conditions, or collection of both. If deriveLeft/deriveRight is set, we would try to derive more conditions for left/right plan.

func (*LogicalJoin) ExtractUsedCols

func (p *LogicalJoin) ExtractUsedCols(parentUsedCols []*expression.Column) (leftCols []*expression.Column, rightCols []*expression.Column)

ExtractUsedCols extracts all the needed columns.

func (*LogicalJoin) GetJoinKeys

func (p *LogicalJoin) GetJoinKeys() (leftKeys, rightKeys []*expression.Column, isNullEQ []bool, hasNullEQ bool)

GetJoinKeys extracts join keys(columns) from EqualConditions. It returns left join keys, right join keys and an `isNullEQ` array which means the `joinKey[i]` is a `NullEQ` function. The `hasNullEQ` means whether there is a `NullEQ` of a join key.

func (*LogicalJoin) GetNAJoinKeys

func (p *LogicalJoin) GetNAJoinKeys() (leftKeys, rightKeys []*expression.Column)

GetNAJoinKeys extracts join keys(columns) from NAEqualCondition.

func (*LogicalJoin) GetPotentialPartitionKeys

func (p *LogicalJoin) GetPotentialPartitionKeys() (leftKeys, rightKeys []*property.MPPPartitionColumn)

GetPotentialPartitionKeys return potential partition keys for join, the potential partition keys are the join keys of EqualConditions

func (*LogicalJoin) Hash64

func (op *LogicalJoin) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalJoin) Init

func (p LogicalJoin) Init(ctx base.PlanContext, offset int) *LogicalJoin

Init initializes LogicalJoin.

func (*LogicalJoin) IsNAAJ

func (p *LogicalJoin) IsNAAJ() bool

IsNAAJ checks if the join is a non-adjacent-join.

func (*LogicalJoin) MergeSchema

func (p *LogicalJoin) MergeSchema()

MergeSchema merge the schema of left and right child of join.

func (*LogicalJoin) PredicatePushDown

func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan)

PredicatePushDown implements the base.LogicalPlan.<1st> interface.

func (*LogicalJoin) PreferAny

func (p *LogicalJoin) PreferAny(joinFlags ...uint) bool

PreferAny checks whether the join type is in the joinFlags.

func (*LogicalJoin) PreparePossibleProperties

func (p *LogicalJoin) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

func (*LogicalJoin) PruneColumns

func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements the base.LogicalPlan.<2nd> interface.

func (*LogicalJoin) PushDownTopN

func (p *LogicalJoin) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PushDownTopN implements the base.LogicalPlan.<5th> interface.

func (*LogicalJoin) ReplaceExprColumns

func (p *LogicalJoin) ReplaceExprColumns(replace map[string]*expression.Column)

ReplaceExprColumns implements base.LogicalPlan interface.

func (*LogicalJoin) SetPreferredJoinType

func (p *LogicalJoin) SetPreferredJoinType()

SetPreferredJoinType generates hint information for the logicalJoin based on the hint information of its left and right children.

func (*LogicalJoin) SetPreferredJoinTypeAndOrder

func (p *LogicalJoin) SetPreferredJoinTypeAndOrder(hintInfo *utilhint.PlanHints)

SetPreferredJoinTypeAndOrder sets the preferred join type and order for the LogicalJoin.

func (*LogicalJoin) Shallow

func (p *LogicalJoin) Shallow() *LogicalJoin

Shallow copies a LogicalJoin struct.

type LogicalLimit

type LogicalLimit struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	PartitionBy      []property.SortItem `hash64-equals:"true"` // This is used for enhanced topN optimization
	Offset           uint64              `hash64-equals:"true"`
	Count            uint64              `hash64-equals:"true"`
	PreferLimitToCop bool
	IsPartial        bool
}

LogicalLimit represents offset and limit plan.

func (*LogicalLimit) BuildKeyInfo

func (p *LogicalLimit) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*LogicalLimit) DeriveStats

func (p *LogicalLimit) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implement base.LogicalPlan.<11th> interface.

func (*LogicalLimit) Equals

func (op *LogicalLimit) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalLimit pointer.

func (*LogicalLimit) ExhaustPhysicalPlans

func (p *LogicalLimit) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalLimit) ExplainInfo

func (p *LogicalLimit) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalLimit) GetPartitionBy

func (p *LogicalLimit) GetPartitionBy() []property.SortItem

GetPartitionBy returns partition by fields

func (*LogicalLimit) Hash64

func (op *LogicalLimit) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (*LogicalLimit) HashCode

func (p *LogicalLimit) HashCode() []byte

HashCode implements LogicalPlan.<0th> interface.

func (LogicalLimit) Init

func (p LogicalLimit) Init(ctx base.PlanContext, offset int) *LogicalLimit

Init initializes LogicalLimit.

func (*LogicalLimit) PredicatePushDown

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalLimit) PruneColumns

func (p *LogicalLimit) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

func (*LogicalLimit) PushDownTopN

func (p *LogicalLimit) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PushDownTopN implements the base.LogicalPlan.<5th> interface.

type LogicalLock

type LogicalLock struct {
	BaseLogicalPlan

	Lock         *ast.SelectLockInfo
	TblID2Handle map[int64][]util.HandleCols

	// tblID2phyTblIDCol is used for partitioned tables,
	// the child executor need to return an extra column containing
	// the Physical Table ID (i.e. from which partition the row came from)
	TblID2PhysTblIDCol map[int64]*expression.Column
}

LogicalLock represents a select lock plan.

func (*LogicalLock) ExhaustPhysicalPlans

func (p *LogicalLock) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (LogicalLock) Init

func (p LogicalLock) Init(ctx base.PlanContext) *LogicalLock

Init initializes LogicalLock.

func (*LogicalLock) PruneColumns

func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

func (*LogicalLock) PushDownTopN

func (p *LogicalLock) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PushDownTopN implements the base.LogicalPlan.<5th> interface.

type LogicalMaxOneRow

type LogicalMaxOneRow struct {
	// logical max one row, doesn't have any other attribute to distinguish, use plan id inside.
	BaseLogicalPlan `hash64-equals:"true"`
}

LogicalMaxOneRow checks if a query returns no more than one row.

func (*LogicalMaxOneRow) DeriveStats

func (p *LogicalMaxOneRow) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements base.LogicalPlan.<11th> interface.

func (*LogicalMaxOneRow) Equals

func (op *LogicalMaxOneRow) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalMaxOneRow pointer.

func (*LogicalMaxOneRow) ExhaustPhysicalPlans

func (p *LogicalMaxOneRow) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalMaxOneRow) Hash64

func (op *LogicalMaxOneRow) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalMaxOneRow) Init

func (p LogicalMaxOneRow) Init(ctx base.PlanContext, offset int) *LogicalMaxOneRow

Init initializes LogicalMaxOneRow.

func (*LogicalMaxOneRow) PredicatePushDown

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalMaxOneRow) Schema

func (p *LogicalMaxOneRow) Schema() *expression.Schema

Schema implements the Plan.Schema interface.

type LogicalMemTable

type LogicalMemTable struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	Extractor base.MemTablePredicateExtractor
	DBName    pmodel.CIStr     `hash64-equals:"true"`
	TableInfo *model.TableInfo `hash64-equals:"true"`
	Columns   []*model.ColumnInfo
	// QueryTimeRange is used to specify the time range for metrics summary tables and inspection tables
	// e.g: select /*+ time_range('2020-02-02 12:10:00', '2020-02-02 13:00:00') */ from metrics_summary;
	//      select /*+ time_range('2020-02-02 12:10:00', '2020-02-02 13:00:00') */ from metrics_summary_by_label;
	//      select /*+ time_range('2020-02-02 12:10:00', '2020-02-02 13:00:00') */ from inspection_summary;
	//      select /*+ time_range('2020-02-02 12:10:00', '2020-02-02 13:00:00') */ from inspection_result;
	QueryTimeRange util.QueryTimeRange
}

LogicalMemTable represents a memory table or virtual table Some memory tables wants to take the ownership of some predications e.g SELECT * FROM cluster_log WHERE type='tikv' AND address='192.16.5.32' Assume that the table `cluster_log` is a memory table, which is used to retrieve logs from remote components. In the above situation we should send log search request to the target TiKV (192.16.5.32) directly instead of requesting all cluster components log search gRPC interface to retrieve log message and filtering them in TiDB node.

func (*LogicalMemTable) DeriveStats

func (p *LogicalMemTable) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements base.LogicalPlan.<11th> interface.

func (*LogicalMemTable) Equals

func (op *LogicalMemTable) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalMemTable pointer.

func (*LogicalMemTable) FindBestTask

func (p *LogicalMemTable) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, opt *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error)

FindBestTask implements the base.LogicalPlan.<3rd> interface.

func (*LogicalMemTable) Hash64

func (op *LogicalMemTable) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalMemTable) Init

func (p LogicalMemTable) Init(ctx base.PlanContext, offset int) *LogicalMemTable

Init initializes LogicalMemTable.

func (*LogicalMemTable) PredicatePushDown

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalMemTable) PruneColumns

func (p *LogicalMemTable) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

type LogicalPartitionUnionAll

type LogicalPartitionUnionAll struct {
	LogicalUnionAll `hash64-equals:"true"`
}

LogicalPartitionUnionAll represents the LogicalUnionAll plan is for partition table.

func (*LogicalPartitionUnionAll) Equals

func (op *LogicalPartitionUnionAll) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalPartitionUnionAll pointer.

func (*LogicalPartitionUnionAll) ExhaustPhysicalPlans

func (p *LogicalPartitionUnionAll) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements LogicalPlan interface.

func (*LogicalPartitionUnionAll) Hash64

func (op *LogicalPartitionUnionAll) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalPartitionUnionAll) Init

Init initializes LogicalPartitionUnionAll.

type LogicalProjection

type LogicalProjection struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	Exprs []expression.Expression `hash64-equals:"true"`

	// CalculateNoDelay indicates this Projection is the root Plan and should be
	// calculated without delay and will not return any result to client.
	// Currently it is "true" only when the current sql query is a "DO" statement.
	// See "https://dev.mysql.com/doc/refman/5.7/en/do.html" for more detail.
	CalculateNoDelay bool `hash64-equals:"true"`

	// Proj4Expand is used for expand to project same column reference, while these
	// col may be filled with null so we couldn't just eliminate this projection itself.
	Proj4Expand bool `hash64-equals:"true"`
}

LogicalProjection represents a select fields plan.

func (*LogicalProjection) AppendExpr

AppendExpr adds the expression to the projection.

func (*LogicalProjection) BuildKeyInfo

func (p *LogicalProjection) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*LogicalProjection) ConvertOuterToInnerJoin

func (p *LogicalProjection) ConvertOuterToInnerJoin(predicates []expression.Expression) base.LogicalPlan

ConvertOuterToInnerJoin implements base.LogicalPlan.<24th> interface.

func (*LogicalProjection) DeriveStats

func (p *LogicalProjection) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, childSchema []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implement base.LogicalPlan.<11th> interface.

func (*LogicalProjection) Equals

func (op *LogicalProjection) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalProjection pointer.

func (*LogicalProjection) ExhaustPhysicalPlans

func (p *LogicalProjection) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalProjection) ExplainInfo

func (p *LogicalProjection) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalProjection) ExtractColGroups

func (p *LogicalProjection) ExtractColGroups(colGroups [][]*expression.Column) [][]*expression.Column

ExtractColGroups implements base.LogicalPlan.<12th> interface.

func (*LogicalProjection) ExtractCorrelatedCols

func (p *LogicalProjection) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.

func (*LogicalProjection) ExtractFD

func (p *LogicalProjection) ExtractFD() *fd.FDSet

ExtractFD implements base.LogicalPlan.<22nd> interface.

func (*LogicalProjection) GetUsedCols

func (p *LogicalProjection) GetUsedCols() (usedCols []*expression.Column)

GetUsedCols extracts all of the Columns used by proj.

func (*LogicalProjection) Hash64

func (op *LogicalProjection) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (*LogicalProjection) HashCode

func (p *LogicalProjection) HashCode() []byte

HashCode implements base.LogicalPlan.<0th> interface.

func (LogicalProjection) Init

func (p LogicalProjection) Init(ctx base.PlanContext, qbOffset int) *LogicalProjection

Init initializes LogicalProjection.

func (*LogicalProjection) PredicatePushDown

func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan)

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalProjection) PreparePossibleProperties

func (p *LogicalProjection) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

func (*LogicalProjection) PruneColumns

func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface. If any expression has SetVar function or Sleep function, we do not prune it.

func (*LogicalProjection) PullUpConstantPredicates

func (p *LogicalProjection) PullUpConstantPredicates() []expression.Expression

PullUpConstantPredicates implements base.LogicalPlan.<9th> interface.

func (*LogicalProjection) PushDownTopN

func (p *LogicalProjection) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PushDownTopN implements base.LogicalPlan.<5th> interface.

func (*LogicalProjection) ReplaceExprColumns

func (p *LogicalProjection) ReplaceExprColumns(replace map[string]*expression.Column)

ReplaceExprColumns implements base.LogicalPlan interface.

func (*LogicalProjection) TryToGetChildProp

func (p *LogicalProjection) TryToGetChildProp(prop *property.PhysicalProperty) (*property.PhysicalProperty, bool)

TryToGetChildProp will check if this sort property can be pushed or not. When a sort column will be replaced by scalar function, we refuse it. When a sort column will be replaced by a constant, we just remove it.

type LogicalSchemaProducer

type LogicalSchemaProducer struct {
	BaseLogicalPlan
	// contains filtered or unexported fields
}

LogicalSchemaProducer stores the schema for the logical plans who can produce schema directly.

func (*LogicalSchemaProducer) BuildKeyInfo

func (s *LogicalSchemaProducer) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements LogicalPlan.BuildKeyInfo interface.

func (*LogicalSchemaProducer) Equals

func (s *LogicalSchemaProducer) Equals(other any) bool

Equals implement HashEquals interface.

func (*LogicalSchemaProducer) Hash64

func (s *LogicalSchemaProducer) Hash64(h base.Hasher)

Hash64 implements HashEquals interface.

func (*LogicalSchemaProducer) InlineProjection

func (s *LogicalSchemaProducer) InlineProjection(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp)

InlineProjection prunes unneeded columns inline an executor.

func (*LogicalSchemaProducer) OutputNames

func (s *LogicalSchemaProducer) OutputNames() types.NameSlice

OutputNames implements the Plan.OutputNames interface.

func (*LogicalSchemaProducer) Schema

Schema implements the Plan.Schema interface.

func (*LogicalSchemaProducer) SetOutputNames

func (s *LogicalSchemaProducer) SetOutputNames(names types.NameSlice)

SetOutputNames sets the output names for the plan.

func (*LogicalSchemaProducer) SetSchema

func (s *LogicalSchemaProducer) SetSchema(schema *expression.Schema)

SetSchema sets the logical schema producer's schema.

func (*LogicalSchemaProducer) SetSchemaAndNames

func (s *LogicalSchemaProducer) SetSchemaAndNames(schema *expression.Schema, names types.NameSlice)

SetSchemaAndNames sets the schema and names for the plan.

type LogicalSelection

type LogicalSelection struct {
	BaseLogicalPlan

	// Originally the WHERE or ON condition is parsed into a single expression,
	// but after we converted to CNF(Conjunctive normal form), it can be
	// split into a list of AND conditions.
	Conditions []expression.Expression `hash64-equals:"true"`
}

LogicalSelection represents a where or having predicate.

func (*LogicalSelection) BuildKeyInfo

func (p *LogicalSelection) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*LogicalSelection) CanPushDown

func (p *LogicalSelection) CanPushDown(storeTp kv.StoreType) bool

CanPushDown is utility function to check whether we can push down Selection to TiKV or TiFlash

func (*LogicalSelection) ConvertOuterToInnerJoin

func (p *LogicalSelection) ConvertOuterToInnerJoin(predicates []expression.Expression) base.LogicalPlan

ConvertOuterToInnerJoin implements base.LogicalPlan.<24th> interface.

func (*LogicalSelection) DeriveStats

func (p *LogicalSelection) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements base.LogicalPlan.<11th> interface.

func (*LogicalSelection) DeriveTopN

DeriveTopN implements the base.LogicalPlan.<6th> interface.

func (*LogicalSelection) Equals

func (op *LogicalSelection) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalSelection pointer.

func (*LogicalSelection) ExhaustPhysicalPlans

func (p *LogicalSelection) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalSelection) ExplainInfo

func (p *LogicalSelection) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalSelection) ExtractCorrelatedCols

func (p *LogicalSelection) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.

func (*LogicalSelection) ExtractFD

func (p *LogicalSelection) ExtractFD() *fd.FDSet

ExtractFD implements the base.LogicalPlan.<22nd> interface.

func (*LogicalSelection) Hash64

func (op *LogicalSelection) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (*LogicalSelection) HashCode

func (p *LogicalSelection) HashCode() []byte

HashCode implements base.LogicalPlan.<0th> interface.

func (LogicalSelection) Init

func (p LogicalSelection) Init(ctx base.PlanContext, qbOffset int) *LogicalSelection

Init initializes LogicalSelection.

func (*LogicalSelection) PredicatePushDown

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalSelection) PreparePossibleProperties

func (*LogicalSelection) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

func (*LogicalSelection) PruneColumns

func (p *LogicalSelection) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

func (*LogicalSelection) PullUpConstantPredicates

func (p *LogicalSelection) PullUpConstantPredicates() []expression.Expression

PullUpConstantPredicates implements the base.LogicalPlan.<9th> interface.

func (*LogicalSelection) ReplaceExprColumns

func (p *LogicalSelection) ReplaceExprColumns(replace map[string]*expression.Column)

ReplaceExprColumns implements base.LogicalPlan interface.

type LogicalSequence

type LogicalSequence struct {
	// logical sequence doesn't have any other attribute to distinguish, use plan id inside.
	BaseLogicalPlan `hash64-equals:"true"`
}

LogicalSequence is used to mark the CTE producer in the main query tree. Its last child is main query. The previous children are cte producers. And there might be dependencies between the CTE producers:

Suppose that the sequence has 4 children, naming c0, c1, c2, c3.
From the definition, c3 is the main query. c0, c1, c2 are CTE producers.
It's possible that c1 references c0, c2 references c1 and c2.
But it's not possible that c0 references c1 or c2.

We use this property to do complex optimizations for CTEs.

func (*LogicalSequence) DeriveStats

func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements the base.LogicalPlan.<11th> interface.

func (*LogicalSequence) Equals

func (op *LogicalSequence) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalSequence pointer.

func (*LogicalSequence) ExhaustPhysicalPlans

func (p *LogicalSequence) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements the base.LogicalPlan.<14th> interface.

func (*LogicalSequence) Hash64

func (op *LogicalSequence) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalSequence) Init

func (p LogicalSequence) Init(ctx base.PlanContext, offset int) *LogicalSequence

Init initializes LogicalSequence

func (*LogicalSequence) PredicatePushDown

PredicatePushDown implements the base.LogicalPlan.<1st> interface. Currently, we only maintain the main query tree.

func (*LogicalSequence) PruneColumns

func (p *LogicalSequence) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements the base.LogicalPlan.<2nd> interface.

func (*LogicalSequence) Schema

func (p *LogicalSequence) Schema() *expression.Schema

Schema returns its last child(which is the main query plan)'s schema.

type LogicalShow

type LogicalShow struct {
	LogicalSchemaProducer `hash64-equals:"true"`
	ShowContents

	Extractor base.ShowPredicateExtractor
}

LogicalShow represents a show plan.

func (*LogicalShow) DeriveStats

func (p *LogicalShow) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implement base.LogicalPlan.<11th> interface.

func (*LogicalShow) Equals

func (op *LogicalShow) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalShow pointer.

func (*LogicalShow) FindBestTask

FindBestTask implements the base.LogicalPlan.<3rd> interface.

func (*LogicalShow) Hash64

func (op *LogicalShow) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalShow) Init

func (p LogicalShow) Init(ctx base.PlanContext) *LogicalShow

Init initializes LogicalShow.

type LogicalShowDDLJobs

type LogicalShowDDLJobs struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	JobNumber int64
}

LogicalShowDDLJobs is for showing DDL job list.

func (*LogicalShowDDLJobs) DeriveStats

func (p *LogicalShowDDLJobs) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements the base.LogicalPlan.<11th> interface.

func (*LogicalShowDDLJobs) Equals

func (op *LogicalShowDDLJobs) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalShowDDLJobs pointer.

func (*LogicalShowDDLJobs) FindBestTask

FindBestTask implements the base.LogicalPlan.<3rd> interface.

func (*LogicalShowDDLJobs) Hash64

func (op *LogicalShowDDLJobs) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalShowDDLJobs) Init

Init initializes LogicalShowDDLJobs.

type LogicalSort

type LogicalSort struct {
	BaseLogicalPlan

	ByItems []*util.ByItems `hash64-equals:"true"`
}

LogicalSort stands for the order by plan.

func (*LogicalSort) Equals

func (op *LogicalSort) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalSort pointer.

func (*LogicalSort) ExhaustPhysicalPlans

func (ls *LogicalSort) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalSort) ExplainInfo

func (ls *LogicalSort) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalSort) ExtractCorrelatedCols

func (ls *LogicalSort) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.

func (*LogicalSort) Hash64

func (op *LogicalSort) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalSort) Init

func (ls LogicalSort) Init(ctx base.PlanContext, offset int) *LogicalSort

Init initializes LogicalSort.

func (*LogicalSort) PreparePossibleProperties

func (ls *LogicalSort) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

func (*LogicalSort) PruneColumns

func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface. If any expression can view as a constant in execution stage, such as correlated column, constant, we do prune them. Note that we can't prune the expressions contain non-deterministic functions, such as rand().

func (*LogicalSort) PushDownTopN

func (ls *LogicalSort) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PushDownTopN implements the base.LogicalPlan.<5th> interface.

func (*LogicalSort) ReplaceExprColumns

func (ls *LogicalSort) ReplaceExprColumns(replace map[string]*expression.Column)

ReplaceExprColumns implements base.LogicalPlan interface.

type LogicalTableDual

type LogicalTableDual struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	// RowCount could only be 0 or 1.
	RowCount int `hash64-equals:"true"`
}

LogicalTableDual represents a dual table plan. Note that sometimes we don't set schema for LogicalTableDual (most notably in buildTableDual()), which means outputting 0/1 row with zero column. This semantic may be different from your expectation sometimes but should not cause any actual problems now.

func (*LogicalTableDual) BuildKeyInfo

func (p *LogicalTableDual) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*LogicalTableDual) DeriveStats

func (p *LogicalTableDual) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implement base.LogicalPlan.<11th> interface.

func (*LogicalTableDual) Equals

func (op *LogicalTableDual) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalTableDual pointer.

func (*LogicalTableDual) ExplainInfo

func (p *LogicalTableDual) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalTableDual) FindBestTask

FindBestTask implements the base.LogicalPlan.<3rd> interface.

func (*LogicalTableDual) Hash64

func (op *LogicalTableDual) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (*LogicalTableDual) HashCode

func (p *LogicalTableDual) HashCode() []byte

HashCode implements base.LogicalPlan.<0th> interface.

func (LogicalTableDual) Init

func (p LogicalTableDual) Init(ctx base.PlanContext, offset int) *LogicalTableDual

Init initializes LogicalTableDual.

func (*LogicalTableDual) PredicatePushDown

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalTableDual) PruneColumns

func (p *LogicalTableDual) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

type LogicalTableScan

type LogicalTableScan struct {
	LogicalSchemaProducer
	Source      *DataSource
	HandleCols  util.HandleCols
	AccessConds expression.CNFExprs
	Ranges      []*ranger.Range
}

LogicalTableScan is the logical table scan operator for TiKV.

func (*LogicalTableScan) BuildKeyInfo

func (ts *LogicalTableScan) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*LogicalTableScan) DeriveStats

func (ts *LogicalTableScan) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (_ *property.StatsInfo, err error)

DeriveStats implements base.LogicalPlan.<11th> interface.

func (*LogicalTableScan) ExplainInfo

func (ts *LogicalTableScan) ExplainInfo() string

ExplainInfo implements Plan interface.

func (LogicalTableScan) Init

func (ts LogicalTableScan) Init(ctx base.PlanContext, offset int) *LogicalTableScan

Init initializes LogicalTableScan.

func (*LogicalTableScan) PreparePossibleProperties

func (ts *LogicalTableScan) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

type LogicalTopN

type LogicalTopN struct {
	BaseLogicalPlan

	ByItems []*util.ByItems `hash64-equals:"true"`
	// PartitionBy is used for extended TopN to consider K heaps. Used by rule_derive_topn_from_window
	PartitionBy      []property.SortItem `hash64-equals:"true"` // This is used for enhanced topN optimization
	Offset           uint64              `hash64-equals:"true"`
	Count            uint64              `hash64-equals:"true"`
	PreferLimitToCop bool                `hash64-equals:"true"`
}

LogicalTopN represents a top-n plan.

func (*LogicalTopN) AttachChild

AttachChild set p as topn's child, for difference with LogicalPlan.SetChild(). AttachChild will tracer the children change while SetChild doesn't.

func (*LogicalTopN) BuildKeyInfo

func (lt *LogicalTopN) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*LogicalTopN) DeriveStats

func (lt *LogicalTopN) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implement base.LogicalPlan.<11th> interface.

func (*LogicalTopN) Equals

func (op *LogicalTopN) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalTopN pointer.

func (*LogicalTopN) ExhaustPhysicalPlans

func (lt *LogicalTopN) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalTopN) ExplainInfo

func (lt *LogicalTopN) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalTopN) ExtractCorrelatedCols

func (lt *LogicalTopN) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.

func (*LogicalTopN) GetPartitionBy

func (lt *LogicalTopN) GetPartitionBy() []property.SortItem

GetPartitionBy returns partition by fields

func (*LogicalTopN) Hash64

func (op *LogicalTopN) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalTopN) Init

func (lt LogicalTopN) Init(ctx base.PlanContext, offset int) *LogicalTopN

Init initializes LogicalTopN.

func (*LogicalTopN) IsLimit

func (lt *LogicalTopN) IsLimit() bool

IsLimit checks if TopN is a limit plan.

func (*LogicalTopN) PreparePossibleProperties

func (lt *LogicalTopN) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

func (*LogicalTopN) PruneColumns

func (lt *LogicalTopN) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface. If any expression can view as a constant in execution stage, such as correlated column, constant, we do prune them. Note that we can't prune the expressions contain non-deterministic functions, such as rand().

func (*LogicalTopN) ReplaceExprColumns

func (lt *LogicalTopN) ReplaceExprColumns(replace map[string]*expression.Column)

ReplaceExprColumns implements base.LogicalPlan interface.

type LogicalUnionAll

type LogicalUnionAll struct {
	LogicalSchemaProducer `hash64-equals:"true"`
}

LogicalUnionAll represents LogicalUnionAll plan.

func (*LogicalUnionAll) DeriveStats

func (p *LogicalUnionAll) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implement base.LogicalPlan.<11th> interface.

func (*LogicalUnionAll) Equals

func (op *LogicalUnionAll) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalUnionAll pointer.

func (*LogicalUnionAll) ExhaustPhysicalPlans

func (p *LogicalUnionAll) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalUnionAll) Hash64

func (op *LogicalUnionAll) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalUnionAll) Init

func (p LogicalUnionAll) Init(ctx base.PlanContext, offset int) *LogicalUnionAll

Init initializes LogicalUnionAll.

func (*LogicalUnionAll) PredicatePushDown

func (p *LogicalUnionAll) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan)

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalUnionAll) PruneColumns

func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

func (*LogicalUnionAll) PushDownTopN

func (p *LogicalUnionAll) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan

PushDownTopN implements the base.LogicalPlan.<5th> interface.

type LogicalUnionScan

type LogicalUnionScan struct {
	BaseLogicalPlan

	Conditions []expression.Expression `hash64-equals:"true"`

	HandleCols util.HandleCols `hash64-equals:"true"`
}

LogicalUnionScan is used in non read-only txn or for scanning a local temporary table whose snapshot data is located in memory.

func (*LogicalUnionScan) Equals

func (op *LogicalUnionScan) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalUnionScan pointer.

func (*LogicalUnionScan) ExhaustPhysicalPlans

func (p *LogicalUnionScan) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalUnionScan) ExplainInfo

func (p *LogicalUnionScan) ExplainInfo() string

ExplainInfo implements Plan interface.

func (*LogicalUnionScan) Hash64

func (op *LogicalUnionScan) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalUnionScan) Init

func (p LogicalUnionScan) Init(ctx base.PlanContext, qbOffset int) *LogicalUnionScan

Init initializes LogicalUnionScan.

func (*LogicalUnionScan) PredicatePushDown

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalUnionScan) PruneColumns

func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

type LogicalWindow

type LogicalWindow struct {
	LogicalSchemaProducer `hash64-equals:"true"`

	WindowFuncDescs []*aggregation.WindowFuncDesc `hash64-equals:"true"`
	PartitionBy     []property.SortItem           `hash64-equals:"true"`
	OrderBy         []property.SortItem           `hash64-equals:"true"`
	Frame           *WindowFrame                  `hash64-equals:"true"`
}

LogicalWindow represents a logical window function plan.

func (*LogicalWindow) CheckComparisonForTiFlash

func (p *LogicalWindow) CheckComparisonForTiFlash(frameBound *FrameBound) bool

CheckComparisonForTiFlash check Duration vs Datetime is invalid comparison as TiFlash can't handle it so far.

func (*LogicalWindow) DeriveStats

func (p *LogicalWindow) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error)

DeriveStats implements base.LogicalPlan.<11th> interface.

func (*LogicalWindow) EqualFrame

func (p *LogicalWindow) EqualFrame(ctx expression.EvalContext, newWindow *LogicalWindow) bool

EqualFrame checks whether two LogicalWindow.Frames are equal.

func (*LogicalWindow) EqualOrderBy

func (p *LogicalWindow) EqualOrderBy(ctx expression.EvalContext, newWindow *LogicalWindow) bool

EqualOrderBy checks whether two LogicalWindow.OrderBys are equal.

func (*LogicalWindow) EqualPartitionBy

func (p *LogicalWindow) EqualPartitionBy(newWindow *LogicalWindow) bool

EqualPartitionBy checks whether two LogicalWindow.Partitions are equal.

func (*LogicalWindow) Equals

func (op *LogicalWindow) Equals(other any) bool

Equals implements the Hash64Equals interface, only receive *LogicalWindow pointer.

func (*LogicalWindow) ExhaustPhysicalPlans

func (p *LogicalWindow) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)

ExhaustPhysicalPlans implements base.LogicalPlan.<14th> interface.

func (*LogicalWindow) ExtractColGroups

func (p *LogicalWindow) ExtractColGroups(colGroups [][]*expression.Column) [][]*expression.Column

ExtractColGroups implements base.LogicalPlan.<12th> interface.

func (*LogicalWindow) ExtractCorrelatedCols

func (p *LogicalWindow) ExtractCorrelatedCols() []*expression.CorrelatedColumn

ExtractCorrelatedCols implements base.LogicalPlan.<15th> interface.

func (*LogicalWindow) GetGroupNDVs

func (*LogicalWindow) GetGroupNDVs(colGroups [][]*expression.Column, childStats []*property.StatsInfo) []property.GroupNDV

GetGroupNDVs gets the GroupNDVs of the LogicalWindow.

func (*LogicalWindow) GetPartitionBy

func (p *LogicalWindow) GetPartitionBy() []property.SortItem

GetPartitionBy returns partition by fields.

func (*LogicalWindow) GetPartitionByCols

func (p *LogicalWindow) GetPartitionByCols() []*expression.Column

GetPartitionByCols extracts 'partition by' columns from the Window.

func (*LogicalWindow) GetPartitionKeys

func (p *LogicalWindow) GetPartitionKeys() []*property.MPPPartitionColumn

GetPartitionKeys gets partition keys for a logical window, it will assign column id for expressions.

func (*LogicalWindow) GetWindowResultColumns

func (p *LogicalWindow) GetWindowResultColumns() []*expression.Column

GetWindowResultColumns returns the columns storing the result of the window function.

func (*LogicalWindow) Hash64

func (op *LogicalWindow) Hash64(h base.Hasher)

Hash64 implements the Hash64Equals interface.

func (LogicalWindow) Init

func (p LogicalWindow) Init(ctx base.PlanContext, offset int) *LogicalWindow

Init initializes LogicalWindow.

func (*LogicalWindow) PredicatePushDown

PredicatePushDown implements base.LogicalPlan.<1st> interface.

func (*LogicalWindow) PreparePossibleProperties

func (p *LogicalWindow) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

func (*LogicalWindow) PruneColumns

func (p *LogicalWindow) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)

PruneColumns implements base.LogicalPlan.<2nd> interface.

func (*LogicalWindow) ReplaceExprColumns

func (p *LogicalWindow) ReplaceExprColumns(replace map[string]*expression.Column)

ReplaceExprColumns implements base.LogicalPlan interface.

type ShowContents

type ShowContents struct {
	Tp                ast.ShowStmtType // Databases/Tables/Columns/....
	DBName            string
	Table             *resolve.TableNameW // Used for showing columns.
	Partition         model.CIStr         // Use for showing partition
	Column            *ast.ColumnName     // Used for `desc table column`.
	IndexName         model.CIStr
	ResourceGroupName string               // Used for showing resource group
	Flag              int                  // Some flag parsed from sql, such as FULL.
	User              *auth.UserIdentity   // Used for show grants.
	Roles             []*auth.RoleIdentity // Used for show grants.

	CountWarningsOrErrors bool // Used for showing count(*) warnings | errors

	Full        bool
	IfNotExists bool       // Used for `show create database if not exists`.
	GlobalScope bool       // Used by show variables.
	Extended    bool       // Used for `show extended columns from ...`
	Limit       *ast.Limit // Used for limit Result Set row number.

	ImportJobID *int64 // Used for SHOW LOAD DATA JOB <jobID>
}

ShowContents stores the contents for the `SHOW` statement.

func (*ShowContents) MemoryUsage

func (s *ShowContents) MemoryUsage() (sum int64)

MemoryUsage return the memory usage of ShowContents

type TiKVSingleGather

type TiKVSingleGather struct {
	LogicalSchemaProducer
	Source *DataSource
	// IsIndexGather marks if this TiKVSingleGather gathers tuples from an IndexScan.
	// in implementation phase, we need this flag to determine whether to generate
	// PhysicalTableReader or PhysicalIndexReader.
	IsIndexGather bool
	Index         *model.IndexInfo
}

TiKVSingleGather is a leaf logical operator of TiDB layer to gather tuples from TiKV regions.

func (*TiKVSingleGather) BuildKeyInfo

func (*TiKVSingleGather) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)

BuildKeyInfo implements base.LogicalPlan.<4th> interface.

func (*TiKVSingleGather) ExplainInfo

func (sg *TiKVSingleGather) ExplainInfo() string

ExplainInfo implements Plan interface.

func (TiKVSingleGather) Init

func (sg TiKVSingleGather) Init(ctx base.PlanContext, offset int) *TiKVSingleGather

Init initializes TiKVSingleGather.

func (*TiKVSingleGather) PreparePossibleProperties

func (*TiKVSingleGather) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column

PreparePossibleProperties implements base.LogicalPlan.<13th> interface.

type WindowFrame

type WindowFrame struct {
	Type  ast.FrameType
	Start *FrameBound
	End   *FrameBound
}

WindowFrame represents a window function frame.

func (*WindowFrame) Clone

func (wf *WindowFrame) Clone() *WindowFrame

Clone copies a window frame totally.

func (*WindowFrame) Equals

func (wf *WindowFrame) Equals(other any) bool

Equals implements HashEquals interface.

func (*WindowFrame) Hash64

func (wf *WindowFrame) Hash64(h base2.Hasher)

Hash64 implements HashEquals interface.

Jump to

Keyboard shortcuts

? : This menu
/ : Search site
f or F : Jump to
y or Y : Canonical URL