Documentation ¶
Index ¶
- func AppendAddSelectionTraceStep(p base.LogicalPlan, child base.LogicalPlan, sel *LogicalSelection, ...)
- func AppendTableDualTraceStep(replaced base.LogicalPlan, dual base.LogicalPlan, ...)
- func BuildLogicalJoinSchema(joinType JoinType, join base.LogicalPlan) *expression.Schema
- func CanPushToCopImpl(lp base.LogicalPlan, storeTp kv.StoreType, considerDual bool) bool
- func Conds2TableDual(p base.LogicalPlan, conds []expression.Expression) base.LogicalPlan
- func DeriveOtherConditions(p *LogicalJoin, leftSchema *expression.Schema, rightSchema *expression.Schema, ...) (leftCond []expression.Expression, rightCond []expression.Expression)
- func HasMaxOneRow(p base.LogicalPlan, childMaxOneRow []bool) bool
- func IsSupportedSelectLockType(lockType ast.SelectLockType) bool
- type BaseLogicalPlan
- func (p *BaseLogicalPlan) BuildKeyInfo(_ *expression.Schema, _ []*expression.Schema)
- func (p *BaseLogicalPlan) BuildPlanTrace() *tracing.PlanTrace
- func (p *BaseLogicalPlan) CanPushToCop(storeTp kv.StoreType) bool
- func (p *BaseLogicalPlan) ChildLen() int
- func (p *BaseLogicalPlan) Children() []base.LogicalPlan
- func (*BaseLogicalPlan) ConstantPropagation(_ base.LogicalPlan, _ int, _ *optimizetrace.LogicalOptimizeOp) (newRoot base.LogicalPlan)
- func (p *BaseLogicalPlan) ConvertOuterToInnerJoin(predicates []expression.Expression) base.LogicalPlan
- func (p *BaseLogicalPlan) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, ...) (*property.StatsInfo, error)
- func (p *BaseLogicalPlan) DeriveTopN(opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- func (p *BaseLogicalPlan) Equals(other any) bool
- func (*BaseLogicalPlan) ExhaustPhysicalPlans(*property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (*BaseLogicalPlan) ExplainInfo() string
- func (*BaseLogicalPlan) ExtractColGroups(_ [][]*expression.Column) [][]*expression.Column
- func (*BaseLogicalPlan) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (p *BaseLogicalPlan) ExtractFD() *fd.FDSet
- func (p *BaseLogicalPlan) FDs() *fd.FDSet
- func (p *BaseLogicalPlan) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, ...) (bestTask base.Task, cntPlan int64, err error)
- func (p *BaseLogicalPlan) GetBaseLogicalPlan() base.LogicalPlan
- func (p *BaseLogicalPlan) GetLogicalTS4TaskMap() uint64
- func (p *BaseLogicalPlan) GetTask(prop *property.PhysicalProperty) base.Task
- func (p *BaseLogicalPlan) Hash64(h base2.Hasher)
- func (p *BaseLogicalPlan) HashCode() []byte
- func (p *BaseLogicalPlan) MaxOneRow() bool
- func (p *BaseLogicalPlan) OutputNames() types.NameSlice
- func (p *BaseLogicalPlan) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (p *BaseLogicalPlan) PredicateSimplification(opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- func (*BaseLogicalPlan) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column
- func (p *BaseLogicalPlan) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (*BaseLogicalPlan) PullUpConstantPredicates() []expression.Expression
- func (p *BaseLogicalPlan) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- func (p *BaseLogicalPlan) RecursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error)
- func (p *BaseLogicalPlan) RollBackTaskMap(ts uint64)
- func (p *BaseLogicalPlan) Schema() *expression.Schema
- func (p *BaseLogicalPlan) Self() base.LogicalPlan
- func (p *BaseLogicalPlan) SetChild(i int, child base.LogicalPlan)
- func (p *BaseLogicalPlan) SetChildren(children ...base.LogicalPlan)
- func (p *BaseLogicalPlan) SetFDs(fd *fd.FDSet)
- func (p *BaseLogicalPlan) SetMaxOneRow(b bool)
- func (p *BaseLogicalPlan) SetOutputNames(names types.NameSlice)
- func (p *BaseLogicalPlan) SetSelf(s base.LogicalPlan)
- func (p *BaseLogicalPlan) StoreTask(prop *property.PhysicalProperty, task base.Task)
- type CTEClass
- type DataSource
- func (ds *DataSource) BuildKeyInfo(selfSchema *expression.Schema, _ []*expression.Schema)
- func (ds *DataSource) Convert2Gathers() (gathers []base.LogicalPlan)
- func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (op *DataSource) Equals(other any) bool
- func (ds *DataSource) ExplainInfo() string
- func (ds *DataSource) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (ds *DataSource) ExtractFD() *fd.FDSet
- func (ds *DataSource) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, ...) (t base.Task, cntPlan int64, err error)
- func (ds *DataSource) GetPKIsHandleCol() *expression.Column
- func (op *DataSource) Hash64(h base.Hasher)
- func (ds DataSource) Init(ctx base.PlanContext, offset int) *DataSource
- func (ds *DataSource) NewExtraHandleSchemaCol() *expression.Column
- func (ds *DataSource) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (ds *DataSource) PredicateSimplification(*optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- func (ds *DataSource) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column
- func (ds *DataSource) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- type FrameBound
- func (fb *FrameBound) Clone() *FrameBound
- func (fb *FrameBound) Equals(other any) bool
- func (fb *FrameBound) Hash64(h base2.Hasher)
- func (fb *FrameBound) ToPB(ctx *base.BuildPBContext) (*tipb.WindowFrameBound, error)
- func (fb *FrameBound) UpdateCmpFuncsAndCmpDataType(cmpDataType types.EvalType)
- func (fb *FrameBound) UpdateCompareCols(ctx sessionctx.Context, orderByCols []*expression.Column) error
- type JoinType
- type LogicalAggregation
- func (la *LogicalAggregation) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (la *LogicalAggregation) BuildSelfKeyInfo(selfSchema *expression.Schema)
- func (la *LogicalAggregation) CanPullUp() bool
- func (la *LogicalAggregation) CanPushToCop(storeTp kv.StoreType) bool
- func (la *LogicalAggregation) CopyAggHints(agg *LogicalAggregation)
- func (la *LogicalAggregation) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, ...) (*property.StatsInfo, error)
- func (la *LogicalAggregation) DistinctArgsMeetsProperty() bool
- func (op *LogicalAggregation) Equals(other any) bool
- func (la *LogicalAggregation) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (la *LogicalAggregation) ExplainInfo() string
- func (la *LogicalAggregation) ExtractColGroups(_ [][]*expression.Column) [][]*expression.Column
- func (la *LogicalAggregation) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (la *LogicalAggregation) ExtractFD() *fd.FDSet
- func (la *LogicalAggregation) GetGroupByCols() []*expression.Column
- func (la *LogicalAggregation) GetPotentialPartitionKeys() []*property.MPPPartitionColumn
- func (la *LogicalAggregation) GetUsedCols() (usedCols []*expression.Column)
- func (la *LogicalAggregation) HasDistinct() bool
- func (la *LogicalAggregation) HasOrderBy() bool
- func (op *LogicalAggregation) Hash64(h base.Hasher)
- func (la LogicalAggregation) Init(ctx base.PlanContext, offset int) *LogicalAggregation
- func (la *LogicalAggregation) IsCompleteModeAgg() bool
- func (la *LogicalAggregation) IsPartialModeAgg() bool
- func (la *LogicalAggregation) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (la *LogicalAggregation) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column
- func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (la *LogicalAggregation) ReplaceExprColumns(replace map[string]*expression.Column)
- func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferStream bool)
- type LogicalApply
- func (la *LogicalApply) CanPullUpAgg() bool
- func (la *LogicalApply) DeCorColFromEqExpr(expr expression.Expression) expression.Expression
- func (la *LogicalApply) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalApply) Equals(other any) bool
- func (la *LogicalApply) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (la *LogicalApply) ExplainInfo() string
- func (la *LogicalApply) ExtractColGroups(colGroups [][]*expression.Column) [][]*expression.Column
- func (la *LogicalApply) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (la *LogicalApply) ExtractFD() *fd.FDSet
- func (op *LogicalApply) Hash64(h base.Hasher)
- func (la LogicalApply) Init(ctx base.PlanContext, offset int) *LogicalApply
- func (la *LogicalApply) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (la *LogicalApply) ReplaceExprColumns(replace map[string]*expression.Column)
- type LogicalCTE
- func (p *LogicalCTE) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (p *LogicalCTE) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p *LogicalCTE) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (p *LogicalCTE) FindBestTask(prop *property.PhysicalProperty, counter *base.PlanCounterTp, ...) (t base.Task, cntPlan int64, err error)
- func (p LogicalCTE) Init(ctx base.PlanContext, offset int) *LogicalCTE
- func (p *LogicalCTE) PredicatePushDown(predicates []expression.Expression, _ *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (p *LogicalCTE) PruneColumns(_ []*expression.Column, _ *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalCTE) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- type LogicalCTETable
- func (p *LogicalCTETable) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (p *LogicalCTETable) FindBestTask(prop *property.PhysicalProperty, _ *base.PlanCounterTp, ...) (t base.Task, cntPlan int64, err error)
- func (p LogicalCTETable) Init(ctx base.PlanContext, offset int) *LogicalCTETable
- type LogicalExpand
- func (op *LogicalExpand) Equals(other any) bool
- func (p *LogicalExpand) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p *LogicalExpand) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (p *LogicalExpand) ExtractFD() *fd.FDSet
- func (p *LogicalExpand) GenLevelProjections()
- func (p *LogicalExpand) GenerateGroupingIDIncrementModeNumericSet(oneSetOffset int) uint64
- func (p *LogicalExpand) GenerateGroupingIDModeBitAnd(oneSet expression.GroupingSet) uint64
- func (p *LogicalExpand) GenerateGroupingMarks(sourceCols []*expression.Column) []map[uint64]struct{}
- func (*LogicalExpand) GetUsedCols() (usedCols []*expression.Column)
- func (op *LogicalExpand) Hash64(h base.Hasher)
- func (p LogicalExpand) Init(ctx base.PlanContext, offset int) *LogicalExpand
- func (p *LogicalExpand) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan)
- func (p *LogicalExpand) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalExpand) ResolveGroupingFuncArgsInGroupBy(groupingFuncArgs []expression.Expression) ([]*expression.Column, error)
- func (p *LogicalExpand) TrySubstituteExprWithGroupingSetCol(expr expression.Expression) (expression.Expression, bool)
- type LogicalIndexScan
- func (is *LogicalIndexScan) BuildKeyInfo(selfSchema *expression.Schema, _ []*expression.Schema)
- func (is *LogicalIndexScan) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (is *LogicalIndexScan) ExplainInfo() string
- func (is *LogicalIndexScan) GetPKIsHandleCol(schema *expression.Schema) *expression.Column
- func (is LogicalIndexScan) Init(ctx base.PlanContext, offset int) *LogicalIndexScan
- func (is *LogicalIndexScan) MatchIndexProp(prop *property.PhysicalProperty) (match bool)
- func (is *LogicalIndexScan) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column
- type LogicalJoin
- func (p *LogicalJoin) AppendJoinConds(eq []*expression.ScalarFunction, left, right, other []expression.Expression)
- func (p *LogicalJoin) AttachOnConds(onConds []expression.Expression)
- func (p *LogicalJoin) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (p *LogicalJoin) ColumnSubstituteAll(schema *expression.Schema, exprs []expression.Expression) (hasFail bool)
- func (p *LogicalJoin) ConstantPropagation(parentPlan base.LogicalPlan, currentChildIdx int, ...) (newRoot base.LogicalPlan)
- func (p *LogicalJoin) ConvertOuterToInnerJoin(predicates []expression.Expression) base.LogicalPlan
- func (p *LogicalJoin) Decorrelate(schema *expression.Schema)
- func (p *LogicalJoin) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalJoin) Equals(other any) bool
- func (p *LogicalJoin) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p *LogicalJoin) ExplainInfo() string
- func (p *LogicalJoin) ExtractColGroups(colGroups [][]*expression.Column) [][]*expression.Column
- func (p *LogicalJoin) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (p *LogicalJoin) ExtractFD() *funcdep.FDSet
- func (p *LogicalJoin) ExtractFDForInnerJoin(filtersFromApply []expression.Expression) *funcdep.FDSet
- func (p *LogicalJoin) ExtractFDForOuterJoin(filtersFromApply []expression.Expression) *funcdep.FDSet
- func (p *LogicalJoin) ExtractFDForSemiJoin(filtersFromApply []expression.Expression) *funcdep.FDSet
- func (p *LogicalJoin) ExtractJoinKeys(childIdx int) *expression.Schema
- func (p *LogicalJoin) ExtractOnCondition(conditions []expression.Expression, leftSchema *expression.Schema, ...) (eqCond []*expression.ScalarFunction, leftCond []expression.Expression, ...)
- func (p *LogicalJoin) ExtractUsedCols(parentUsedCols []*expression.Column) (leftCols []*expression.Column, rightCols []*expression.Column)
- func (p *LogicalJoin) GetJoinKeys() (leftKeys, rightKeys []*expression.Column, isNullEQ []bool, hasNullEQ bool)
- func (p *LogicalJoin) GetNAJoinKeys() (leftKeys, rightKeys []*expression.Column)
- func (p *LogicalJoin) GetPotentialPartitionKeys() (leftKeys, rightKeys []*property.MPPPartitionColumn)
- func (op *LogicalJoin) Hash64(h base.Hasher)
- func (p LogicalJoin) Init(ctx base.PlanContext, offset int) *LogicalJoin
- func (p *LogicalJoin) IsNAAJ() bool
- func (p *LogicalJoin) MergeSchema()
- func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan)
- func (p *LogicalJoin) PreferAny(joinFlags ...uint) bool
- func (p *LogicalJoin) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column
- func (p *LogicalJoin) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalJoin) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- func (p *LogicalJoin) ReplaceExprColumns(replace map[string]*expression.Column)
- func (p *LogicalJoin) SetPreferredJoinType()
- func (p *LogicalJoin) SetPreferredJoinTypeAndOrder(hintInfo *utilhint.PlanHints)
- func (p *LogicalJoin) Shallow() *LogicalJoin
- type LogicalLimit
- func (p *LogicalLimit) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (p *LogicalLimit) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalLimit) Equals(other any) bool
- func (p *LogicalLimit) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p *LogicalLimit) ExplainInfo() string
- func (p *LogicalLimit) GetPartitionBy() []property.SortItem
- func (op *LogicalLimit) Hash64(h base.Hasher)
- func (p *LogicalLimit) HashCode() []byte
- func (p LogicalLimit) Init(ctx base.PlanContext, offset int) *LogicalLimit
- func (p *LogicalLimit) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (p *LogicalLimit) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalLimit) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- type LogicalLock
- func (p *LogicalLock) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p LogicalLock) Init(ctx base.PlanContext) *LogicalLock
- func (p *LogicalLock) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalLock) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- type LogicalMaxOneRow
- func (p *LogicalMaxOneRow) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalMaxOneRow) Equals(other any) bool
- func (p *LogicalMaxOneRow) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (op *LogicalMaxOneRow) Hash64(h base.Hasher)
- func (p LogicalMaxOneRow) Init(ctx base.PlanContext, offset int) *LogicalMaxOneRow
- func (p *LogicalMaxOneRow) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (p *LogicalMaxOneRow) Schema() *expression.Schema
- type LogicalMemTable
- func (p *LogicalMemTable) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalMemTable) Equals(other any) bool
- func (p *LogicalMemTable) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, ...) (t base.Task, cntPlan int64, err error)
- func (op *LogicalMemTable) Hash64(h base.Hasher)
- func (p LogicalMemTable) Init(ctx base.PlanContext, offset int) *LogicalMemTable
- func (p *LogicalMemTable) PredicatePushDown(predicates []expression.Expression, _ *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (p *LogicalMemTable) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- type LogicalPartitionUnionAll
- func (op *LogicalPartitionUnionAll) Equals(other any) bool
- func (p *LogicalPartitionUnionAll) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (op *LogicalPartitionUnionAll) Hash64(h base.Hasher)
- func (p LogicalPartitionUnionAll) Init(ctx base.PlanContext, offset int) *LogicalPartitionUnionAll
- type LogicalProjection
- func (p *LogicalProjection) AppendExpr(expr expression.Expression) *expression.Column
- func (p *LogicalProjection) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (p *LogicalProjection) ConvertOuterToInnerJoin(predicates []expression.Expression) base.LogicalPlan
- func (p *LogicalProjection) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalProjection) Equals(other any) bool
- func (p *LogicalProjection) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p *LogicalProjection) ExplainInfo() string
- func (p *LogicalProjection) ExtractColGroups(colGroups [][]*expression.Column) [][]*expression.Column
- func (p *LogicalProjection) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (p *LogicalProjection) ExtractFD() *fd.FDSet
- func (p *LogicalProjection) GetUsedCols() (usedCols []*expression.Column)
- func (op *LogicalProjection) Hash64(h base.Hasher)
- func (p *LogicalProjection) HashCode() []byte
- func (p LogicalProjection) Init(ctx base.PlanContext, qbOffset int) *LogicalProjection
- func (p *LogicalProjection) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan)
- func (p *LogicalProjection) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column
- func (p *LogicalProjection) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalProjection) PullUpConstantPredicates() []expression.Expression
- func (p *LogicalProjection) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- func (p *LogicalProjection) ReplaceExprColumns(replace map[string]*expression.Column)
- func (p *LogicalProjection) TryToGetChildProp(prop *property.PhysicalProperty) (*property.PhysicalProperty, bool)
- type LogicalSchemaProducer
- func (s *LogicalSchemaProducer) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (s *LogicalSchemaProducer) Equals(other any) bool
- func (s *LogicalSchemaProducer) Hash64(h base.Hasher)
- func (s *LogicalSchemaProducer) InlineProjection(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp)
- func (s *LogicalSchemaProducer) OutputNames() types.NameSlice
- func (s *LogicalSchemaProducer) Schema() *expression.Schema
- func (s *LogicalSchemaProducer) SetOutputNames(names types.NameSlice)
- func (s *LogicalSchemaProducer) SetSchema(schema *expression.Schema)
- func (s *LogicalSchemaProducer) SetSchemaAndNames(schema *expression.Schema, names types.NameSlice)
- type LogicalSelection
- func (p *LogicalSelection) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (p *LogicalSelection) CanPushDown(storeTp kv.StoreType) bool
- func (p *LogicalSelection) ConvertOuterToInnerJoin(predicates []expression.Expression) base.LogicalPlan
- func (p *LogicalSelection) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (p *LogicalSelection) DeriveTopN(opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- func (op *LogicalSelection) Equals(other any) bool
- func (p *LogicalSelection) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p *LogicalSelection) ExplainInfo() string
- func (p *LogicalSelection) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (p *LogicalSelection) ExtractFD() *fd.FDSet
- func (op *LogicalSelection) Hash64(h base.Hasher)
- func (p *LogicalSelection) HashCode() []byte
- func (p LogicalSelection) Init(ctx base.PlanContext, qbOffset int) *LogicalSelection
- func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (*LogicalSelection) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column
- func (p *LogicalSelection) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalSelection) PullUpConstantPredicates() []expression.Expression
- func (p *LogicalSelection) ReplaceExprColumns(replace map[string]*expression.Column)
- type LogicalSequence
- func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (p *LogicalSequence) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p LogicalSequence) Init(ctx base.PlanContext, offset int) *LogicalSequence
- func (p *LogicalSequence) PredicatePushDown(predicates []expression.Expression, op *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (p *LogicalSequence) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalSequence) Schema() *expression.Schema
- type LogicalShow
- func (p *LogicalShow) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalShow) Equals(other any) bool
- func (p *LogicalShow) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, ...) (base.Task, int64, error)
- func (op *LogicalShow) Hash64(h base.Hasher)
- func (p LogicalShow) Init(ctx base.PlanContext) *LogicalShow
- type LogicalShowDDLJobs
- func (p *LogicalShowDDLJobs) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalShowDDLJobs) Equals(other any) bool
- func (p *LogicalShowDDLJobs) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, ...) (base.Task, int64, error)
- func (op *LogicalShowDDLJobs) Hash64(h base.Hasher)
- func (p LogicalShowDDLJobs) Init(ctx base.PlanContext) *LogicalShowDDLJobs
- type LogicalSort
- func (op *LogicalSort) Equals(other any) bool
- func (ls *LogicalSort) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (ls *LogicalSort) ExplainInfo() string
- func (ls *LogicalSort) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (op *LogicalSort) Hash64(h base.Hasher)
- func (ls LogicalSort) Init(ctx base.PlanContext, offset int) *LogicalSort
- func (ls *LogicalSort) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column
- func (ls *LogicalSort) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (ls *LogicalSort) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- func (ls *LogicalSort) ReplaceExprColumns(replace map[string]*expression.Column)
- type LogicalTableDual
- func (p *LogicalTableDual) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (p *LogicalTableDual) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalTableDual) Equals(other any) bool
- func (p *LogicalTableDual) ExplainInfo() string
- func (p *LogicalTableDual) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, ...) (base.Task, int64, error)
- func (op *LogicalTableDual) Hash64(h base.Hasher)
- func (p *LogicalTableDual) HashCode() []byte
- func (p LogicalTableDual) Init(ctx base.PlanContext, offset int) *LogicalTableDual
- func (p *LogicalTableDual) PredicatePushDown(predicates []expression.Expression, _ *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (p *LogicalTableDual) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- type LogicalTableScan
- func (ts *LogicalTableScan) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (ts *LogicalTableScan) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, ...) (_ *property.StatsInfo, err error)
- func (ts *LogicalTableScan) ExplainInfo() string
- func (ts LogicalTableScan) Init(ctx base.PlanContext, offset int) *LogicalTableScan
- func (ts *LogicalTableScan) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column
- type LogicalTopN
- func (lt *LogicalTopN) AttachChild(p base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- func (lt *LogicalTopN) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (lt *LogicalTopN) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalTopN) Equals(other any) bool
- func (lt *LogicalTopN) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (lt *LogicalTopN) ExplainInfo() string
- func (lt *LogicalTopN) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (lt *LogicalTopN) GetPartitionBy() []property.SortItem
- func (op *LogicalTopN) Hash64(h base.Hasher)
- func (lt LogicalTopN) Init(ctx base.PlanContext, offset int) *LogicalTopN
- func (lt *LogicalTopN) IsLimit() bool
- func (lt *LogicalTopN) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column
- func (lt *LogicalTopN) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (lt *LogicalTopN) ReplaceExprColumns(replace map[string]*expression.Column)
- type LogicalUnionAll
- func (p *LogicalUnionAll) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, ...) (*property.StatsInfo, error)
- func (op *LogicalUnionAll) Equals(other any) bool
- func (p *LogicalUnionAll) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (op *LogicalUnionAll) Hash64(h base.Hasher)
- func (p LogicalUnionAll) Init(ctx base.PlanContext, offset int) *LogicalUnionAll
- func (p *LogicalUnionAll) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) (ret []expression.Expression, retPlan base.LogicalPlan)
- func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalUnionAll) PushDownTopN(topNLogicalPlan base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
- type LogicalUnionScan
- func (op *LogicalUnionScan) Equals(other any) bool
- func (p *LogicalUnionScan) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p *LogicalUnionScan) ExplainInfo() string
- func (op *LogicalUnionScan) Hash64(h base.Hasher)
- func (p LogicalUnionScan) Init(ctx base.PlanContext, qbOffset int) *LogicalUnionScan
- func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (p *LogicalUnionScan) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- type LogicalWindow
- func (p *LogicalWindow) CheckComparisonForTiFlash(frameBound *FrameBound) bool
- func (p *LogicalWindow) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, ...) (*property.StatsInfo, error)
- func (p *LogicalWindow) EqualFrame(ctx expression.EvalContext, newWindow *LogicalWindow) bool
- func (p *LogicalWindow) EqualOrderBy(ctx expression.EvalContext, newWindow *LogicalWindow) bool
- func (p *LogicalWindow) EqualPartitionBy(newWindow *LogicalWindow) bool
- func (op *LogicalWindow) Equals(other any) bool
- func (p *LogicalWindow) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
- func (p *LogicalWindow) ExtractColGroups(colGroups [][]*expression.Column) [][]*expression.Column
- func (p *LogicalWindow) ExtractCorrelatedCols() []*expression.CorrelatedColumn
- func (*LogicalWindow) GetGroupNDVs(colGroups [][]*expression.Column, childStats []*property.StatsInfo) []property.GroupNDV
- func (p *LogicalWindow) GetPartitionBy() []property.SortItem
- func (p *LogicalWindow) GetPartitionByCols() []*expression.Column
- func (p *LogicalWindow) GetPartitionKeys() []*property.MPPPartitionColumn
- func (p *LogicalWindow) GetWindowResultColumns() []*expression.Column
- func (op *LogicalWindow) Hash64(h base.Hasher)
- func (p LogicalWindow) Init(ctx base.PlanContext, offset int) *LogicalWindow
- func (p *LogicalWindow) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
- func (p *LogicalWindow) PreparePossibleProperties(_ *expression.Schema, _ ...[][]*expression.Column) [][]*expression.Column
- func (p *LogicalWindow) PruneColumns(parentUsedCols []*expression.Column, opt *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
- func (p *LogicalWindow) ReplaceExprColumns(replace map[string]*expression.Column)
- type ShowContents
- type TiKVSingleGather
- func (*TiKVSingleGather) BuildKeyInfo(selfSchema *expression.Schema, childSchema []*expression.Schema)
- func (sg *TiKVSingleGather) ExplainInfo() string
- func (sg TiKVSingleGather) Init(ctx base.PlanContext, offset int) *TiKVSingleGather
- func (*TiKVSingleGather) PreparePossibleProperties(_ *expression.Schema, childrenProperties ...[][]*expression.Column) [][]*expression.Column
- type WindowFrame
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 ¶
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 ¶
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 ¶
func (p *BaseLogicalPlan) DeriveTopN(opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
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 ¶
func (p *BaseLogicalPlan) GetTask(prop *property.PhysicalProperty) base.Task
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 ¶
func (p *BaseLogicalPlan) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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 ¶
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 ¶
IsOuterJoin returns if this joiner is an outer joiner
func (JoinType) IsSemiJoin ¶
IsSemiJoin returns if this joiner is a semi/anti-semi joiner
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 ¶
func (la LogicalAggregation) Init(ctx base.PlanContext, offset int) *LogicalAggregation
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 ¶
func (la *LogicalAggregation) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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 ¶
func (p *LogicalCTE) PredicatePushDown(predicates []expression.Expression, _ *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
PredicatePushDown implements base.LogicalPlan.<1st> interface.
func (*LogicalCTE) PruneColumns ¶
func (p *LogicalCTE) PruneColumns(_ []*expression.Column, _ *optimizetrace.LogicalOptimizeOp) (base.LogicalPlan, error)
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 ¶
func (p *LogicalCTETable) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error)
DeriveStats implements the base.LogicalPlan.<11th> interface.
func (*LogicalCTETable) FindBestTask ¶
func (p *LogicalCTETable) FindBestTask(prop *property.PhysicalProperty, _ *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (t base.Task, cntPlan int64, err error)
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 | +------------+
- '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
- '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 ¶
func (p *LogicalLimit) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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 {
BaseLogicalPlan
}
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 ¶
func (p *LogicalMaxOneRow) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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 ¶
func (p *LogicalMemTable) PredicatePushDown(predicates []expression.Expression, _ *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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 ¶
func (p LogicalPartitionUnionAll) Init(ctx base.PlanContext, offset int) *LogicalPartitionUnionAll
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 ¶
func (p *LogicalProjection) AppendExpr(expr expression.Expression) *expression.Column
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 ¶
func (s *LogicalSchemaProducer) Schema() *expression.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 ¶
func (p *LogicalSelection) DeriveTopN(opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
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 ¶
func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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 {
BaseLogicalPlan
}
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) ExhaustPhysicalPlans ¶
func (p *LogicalSequence) ExhaustPhysicalPlans(prop *property.PhysicalProperty) ([]base.PhysicalPlan, bool, error)
ExhaustPhysicalPlans implements the base.LogicalPlan.<14th> interface.
func (LogicalSequence) Init ¶
func (p LogicalSequence) Init(ctx base.PlanContext, offset int) *LogicalSequence
Init initializes LogicalSequence
func (*LogicalSequence) PredicatePushDown ¶
func (p *LogicalSequence) PredicatePushDown(predicates []expression.Expression, op *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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 ¶
func (p *LogicalShow) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (base.Task, int64, error)
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 ¶
func (p *LogicalShowDDLJobs) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, _ *optimizetrace.PhysicalOptimizeOp) (base.Task, int64, error)
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 ¶
func (p LogicalShowDDLJobs) Init(ctx base.PlanContext) *LogicalShowDDLJobs
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 ¶
func (p *LogicalTableDual) FindBestTask(prop *property.PhysicalProperty, planCounter *base.PlanCounterTp, opt *optimizetrace.PhysicalOptimizeOp) (base.Task, int64, error)
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 ¶
func (p *LogicalTableDual) PredicatePushDown(predicates []expression.Expression, _ *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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 ¶
func (lt *LogicalTopN) AttachChild(p base.LogicalPlan, opt *optimizetrace.LogicalOptimizeOp) base.LogicalPlan
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 ¶
func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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 ¶
func (p *LogicalWindow) PredicatePushDown(predicates []expression.Expression, opt *optimizetrace.LogicalOptimizeOp) ([]expression.Expression, base.LogicalPlan)
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.
Source Files ¶
- base_logical_plan.go
- hash64_equals_generated.go
- logical_aggregation.go
- logical_apply.go
- logical_cte.go
- logical_cte_table.go
- logical_datasource.go
- logical_expand.go
- logical_index_scan.go
- logical_join.go
- logical_limit.go
- logical_lock.go
- logical_max_one_row.go
- logical_mem_table.go
- logical_partition_union_all.go
- logical_plans_misc.go
- logical_projection.go
- logical_schema_producer.go
- logical_selection.go
- logical_sequence.go
- logical_show.go
- logical_show_ddl_jobs.go
- logical_sort.go
- logical_table_dual.go
- logical_table_scan.go
- logical_tikv_single_gather.go
- logical_top_n.go
- logical_union_all.go
- logical_union_scan.go
- logical_window.go