diff --git a/executor/builder.go b/executor/builder.go index a123ad5ccdac8..46999214cb9b3 100644 --- a/executor/builder.go +++ b/executor/builder.go @@ -5051,7 +5051,7 @@ func (b *executorBuilder) buildShuffle(v *plannercore.PhysicalShuffle) *ShuffleE for _, dataSource := range v.DataSources { stub := plannercore.PhysicalShuffleReceiverStub{ DataSource: dataSource, - }.Init(b.ctx, dataSource.Stats(), dataSource.SelectBlockOffset(), nil) + }.Init(b.ctx, dataSource.StatsInfo(), dataSource.SelectBlockOffset(), nil) stub.SetSchema(dataSource.Schema()) stubs = append(stubs, stub) } diff --git a/planner/cascades/enforcer_rules.go b/planner/cascades/enforcer_rules.go index dcc3a002d6c6d..def263c5fcbc4 100644 --- a/planner/cascades/enforcer_rules.go +++ b/planner/cascades/enforcer_rules.go @@ -65,7 +65,7 @@ func (*OrderEnforcer) OnEnforce(reqProp *property.PhysicalProperty, child memo.I childPlan := child.GetPlan() sort := plannercore.PhysicalSort{ ByItems: make([]*util.ByItems, 0, len(reqProp.SortItems)), - }.Init(childPlan.SCtx(), childPlan.Stats(), childPlan.SelectBlockOffset(), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64}) + }.Init(childPlan.SCtx(), childPlan.StatsInfo(), childPlan.SelectBlockOffset(), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64}) for _, item := range reqProp.SortItems { item := &util.ByItems{ Expr: item.Col, diff --git a/planner/core/BUILD.bazel b/planner/core/BUILD.bazel index e4f110dd6d0d6..977e919162cbd 100644 --- a/planner/core/BUILD.bazel +++ b/planner/core/BUILD.bazel @@ -107,6 +107,7 @@ go_library( "//parser/terror", "//parser/types", "//planner/core/internal", + "//planner/core/internal/base", "//planner/core/metrics", "//planner/funcdep", "//planner/property", diff --git a/planner/core/common_plans.go b/planner/core/common_plans.go index 3dc784172d40f..15b8105eef97e 100644 --- a/planner/core/common_plans.go +++ b/planner/core/common_plans.go @@ -934,7 +934,7 @@ func (e *Explain) RenderResult() error { e.Rows = append(e.Rows, []string{hint.RestoreOptimizerHints(hints)}) case types.ExplainFormatBinary: flat := FlattenPhysicalPlan(e.TargetPlan, false) - str := BinaryPlanStrFromFlatPlan(e.ctx, flat) + str := BinaryPlanStrFromFlatPlan(e.SCtx(), flat) e.Rows = append(e.Rows, []string{str}) case types.ExplainFormatTiDBJSON: flat := FlattenPhysicalPlan(e.TargetPlan, true) @@ -1096,7 +1096,7 @@ func (e *Explain) prepareOperatorInfo(p Plan, taskType, id string) { if strings.ToLower(e.Format) == types.ExplainFormatTrueCardCost || strings.ToLower(e.Format) == types.ExplainFormatCostTrace { row = append(row, costFormula) } - actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfoStr(e.ctx, p, e.RuntimeStatsColl) + actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfoStr(e.SCtx(), p, e.RuntimeStatsColl) row = append(row, actRows, taskType, accessObject, analyzeInfo, operatorInfo, memoryInfo, diskInfo) } else { row = []string{id, estRows} @@ -1128,7 +1128,7 @@ func (e *Explain) prepareOperatorInfoForJSONFormat(p Plan, taskType, id string, } if e.Analyze || e.RuntimeStatsColl != nil { - jsonRow.ActRows, jsonRow.ExecuteInfo, jsonRow.MemoryInfo, jsonRow.DiskInfo = getRuntimeInfoStr(e.ctx, p, e.RuntimeStatsColl) + jsonRow.ActRows, jsonRow.ExecuteInfo, jsonRow.MemoryInfo, jsonRow.DiskInfo = getRuntimeInfoStr(e.SCtx(), p, e.RuntimeStatsColl) } return jsonRow } @@ -1150,7 +1150,7 @@ func (e *Explain) getOperatorInfo(p Plan, id string) (estRows, estCost, costForm costFormula = "N/A" if isPhysicalPlan { estRows = strconv.FormatFloat(pp.getEstRowCountForDisplay(), 'f', 2, 64) - if e.ctx != nil && e.ctx.GetSessionVars().CostModelVersion == modelVer2 { + if e.SCtx() != nil && e.SCtx().GetSessionVars().CostModelVersion == modelVer2 { costVer2, _ := pp.getPlanCostVer2(property.RootTaskType, NewDefaultPlanCostOption()) estCost = strconv.FormatFloat(costVer2.cost, 'f', 2, 64) if costVer2.trace != nil { @@ -1160,7 +1160,7 @@ func (e *Explain) getOperatorInfo(p Plan, id string) (estRows, estCost, costForm planCost, _ := getPlanCost(pp, property.RootTaskType, NewDefaultPlanCostOption()) estCost = strconv.FormatFloat(planCost, 'f', 2, 64) } - } else if si := p.statsInfo(); si != nil { + } else if si := p.StatsInfo(); si != nil { estRows = strconv.FormatFloat(si.RowCount, 'f', 2, 64) } @@ -1168,8 +1168,8 @@ func (e *Explain) getOperatorInfo(p Plan, id string) (estRows, estCost, costForm accessObject = plan.AccessObject().String() operatorInfo = plan.OperatorInfo(false) } else { - if pa, ok := p.(partitionAccesser); ok && e.ctx != nil { - accessObject = pa.accessObject(e.ctx).String() + if pa, ok := p.(partitionAccesser); ok && e.SCtx() != nil { + accessObject = pa.accessObject(e.SCtx()).String() } operatorInfo = p.ExplainInfo() } @@ -1263,7 +1263,7 @@ func binaryOpFromFlatOp(explainCtx sessionctx.Context, op *FlatOperator, out *ti p := op.Origin.(PhysicalPlan) out.Cost, _ = getPlanCost(p, property.RootTaskType, NewDefaultPlanCostOption()) out.EstRows = p.getEstRowCountForDisplay() - } else if statsInfo := op.Origin.statsInfo(); statsInfo != nil { + } else if statsInfo := op.Origin.StatsInfo(); statsInfo != nil { out.EstRows = statsInfo.RowCount } diff --git a/planner/core/encode.go b/planner/core/encode.go index 0d96bfa437c99..9e118ea22bfc2 100644 --- a/planner/core/encode.go +++ b/planner/core/encode.go @@ -43,7 +43,7 @@ func EncodeFlatPlan(flat *FlatPhysicalPlan) string { failpoint.Inject("mockPlanRowCount", func(val failpoint.Value) { selectPlan, _ := flat.Main.GetSelectPlan() for _, op := range selectPlan { - op.Origin.statsInfo().RowCount = float64(val.(int)) + op.Origin.StatsInfo().RowCount = float64(val.(int)) } }) pn := encoderPool.Get().(*planEncoder) @@ -71,7 +71,7 @@ func EncodeFlatPlan(flat *FlatPhysicalPlan) string { var estRows float64 if op.IsPhysicalPlan { estRows = op.Origin.(PhysicalPlan).getEstRowCountForDisplay() - } else if statsInfo := p.statsInfo(); statsInfo != nil { + } else if statsInfo := p.StatsInfo(); statsInfo != nil { estRows = statsInfo.RowCount } plancodec.EncodePlanNode( @@ -103,7 +103,7 @@ func encodeFlatPlanTree(flatTree FlatPlanTree, offset int, buf *bytes.Buffer) { var estRows float64 if op.IsPhysicalPlan { estRows = op.Origin.(PhysicalPlan).getEstRowCountForDisplay() - } else if statsInfo := p.statsInfo(); statsInfo != nil { + } else if statsInfo := p.StatsInfo(); statsInfo != nil { estRows = statsInfo.RowCount } plancodec.EncodePlanNode( @@ -164,7 +164,7 @@ func EncodePlan(p Plan) string { selectPlan := getSelectPlan(p) if selectPlan != nil { failpoint.Inject("mockPlanRowCount", func(val failpoint.Value) { - selectPlan.statsInfo().RowCount = float64(val.(int)) + selectPlan.StatsInfo().RowCount = float64(val.(int)) }) } return pn.encodePlanTree(p) @@ -193,8 +193,8 @@ func (pn *planEncoder) encodeCTEPlan() { taskTypeInfo := plancodec.EncodeTaskType(true, kv.TiKV) actRows, analyzeInfo, memoryInfo, diskInfo := getRuntimeInfoStr(x.SCtx(), x, nil) rowCount := 0.0 - if statsInfo := x.statsInfo(); statsInfo != nil { - rowCount = x.statsInfo().RowCount + if statsInfo := x.StatsInfo(); statsInfo != nil { + rowCount = x.StatsInfo().RowCount } plancodec.EncodePlanNode(0, strconv.Itoa(x.CTE.IDForStorage), plancodec.TypeCTEDefinition, rowCount, taskTypeInfo, x.ExplainInfo(), actRows, analyzeInfo, memoryInfo, diskInfo, &pn.buf) pn.encodePlan(x.SeedPlan, true, kv.TiKV, 1) @@ -211,7 +211,7 @@ func (pn *planEncoder) encodePlan(p Plan, isRoot bool, store kv.StoreType, depth rowCount := 0.0 if pp, ok := p.(PhysicalPlan); ok { rowCount = pp.getEstRowCountForDisplay() - } else if statsInfo := p.statsInfo(); statsInfo != nil { + } else if statsInfo := p.StatsInfo(); statsInfo != nil { rowCount = statsInfo.RowCount } plancodec.EncodePlanNode(depth, strconv.Itoa(p.ID()), p.TP(), rowCount, taskTypeInfo, p.ExplainInfo(), actRows, analyzeInfo, memoryInfo, diskInfo, &pn.buf) diff --git a/planner/core/exhaust_physical_plans.go b/planner/core/exhaust_physical_plans.go index d80a84295658d..263ce5c36becc 100644 --- a/planner/core/exhaust_physical_plans.go +++ b/planner/core/exhaust_physical_plans.go @@ -59,7 +59,7 @@ func (p *LogicalUnionScan) exhaustPhysicalPlans(prop *property.PhysicalProperty) us := PhysicalUnionScan{ Conditions: p.conditions, HandleCols: p.handleCols, - }.Init(p.ctx, p.stats, p.blockOffset, childProp) + }.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), childProp) return []PhysicalPlan{us}, true, nil } @@ -214,7 +214,7 @@ func (p *LogicalJoin) GetMergeJoin(prop *property.PhysicalProperty, schema *expr RightJoinKeys: rightKeys, IsNullEQ: newIsNullEQ, } - mergeJoin := PhysicalMergeJoin{basePhysicalJoin: baseJoin}.Init(p.ctx, statsInfo.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset) + mergeJoin := PhysicalMergeJoin{basePhysicalJoin: baseJoin}.Init(p.SCtx(), statsInfo.ScaleByExpectCnt(prop.ExpectedCnt), p.SelectBlockOffset()) mergeJoin.SetSchema(schema) mergeJoin.OtherConditions = p.moveEqualToOtherConditions(offsets) mergeJoin.initCompareFuncs() @@ -299,11 +299,11 @@ func (p *LogicalJoin) getEnforcedMergeJoin(prop *property.PhysicalProperty, sche isExist, hasLeftColInProp, hasRightColInProp := false, false, false for joinKeyPos := 0; joinKeyPos < len(leftJoinKeys); joinKeyPos++ { var key *expression.Column - if item.Col.Equal(p.ctx, leftJoinKeys[joinKeyPos]) { + if item.Col.Equal(p.SCtx(), leftJoinKeys[joinKeyPos]) { key = leftJoinKeys[joinKeyPos] hasLeftColInProp = true } - if item.Col.Equal(p.ctx, rightJoinKeys[joinKeyPos]) { + if item.Col.Equal(p.SCtx(), rightJoinKeys[joinKeyPos]) { key = rightJoinKeys[joinKeyPos] hasRightColInProp = true } @@ -359,7 +359,7 @@ func (p *LogicalJoin) getEnforcedMergeJoin(prop *property.PhysicalProperty, sche IsNullEQ: newNullEQ, OtherConditions: otherConditions, } - enforcedPhysicalMergeJoin := PhysicalMergeJoin{basePhysicalJoin: baseJoin, Desc: desc}.Init(p.ctx, statsInfo.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset) + enforcedPhysicalMergeJoin := PhysicalMergeJoin{basePhysicalJoin: baseJoin, Desc: desc}.Init(p.SCtx(), statsInfo.ScaleByExpectCnt(prop.ExpectedCnt), p.SelectBlockOffset()) enforcedPhysicalMergeJoin.SetSchema(schema) enforcedPhysicalMergeJoin.childrenReqProps = []*property.PhysicalProperty{lProp, rProp} enforcedPhysicalMergeJoin.initCompareFuncs() @@ -369,7 +369,7 @@ func (p *LogicalJoin) getEnforcedMergeJoin(prop *property.PhysicalProperty, sche func (p *PhysicalMergeJoin) initCompareFuncs() { p.CompareFuncs = make([]expression.CompareFunc, 0, len(p.LeftJoinKeys)) for i := range p.LeftJoinKeys { - p.CompareFuncs = append(p.CompareFuncs, expression.GetCmpFunction(p.ctx, p.LeftJoinKeys[i], p.RightJoinKeys[i])) + p.CompareFuncs = append(p.CompareFuncs, expression.GetCmpFunction(p.SCtx(), p.LeftJoinKeys[i], p.RightJoinKeys[i])) } } @@ -388,7 +388,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy forceLeftToBuild := ((p.preferJoinType & preferLeftAsHJBuild) > 0) || ((p.preferJoinType & preferRightAsHJProbe) > 0) forceRightToBuild := ((p.preferJoinType & preferRightAsHJBuild) > 0) || ((p.preferJoinType & preferLeftAsHJProbe) > 0) if forceLeftToBuild && forceRightToBuild { - p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints")) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints")) forceLeftToBuild = false forceRightToBuild = false } @@ -399,7 +399,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy joins = append(joins, p.getHashJoin(prop, 1, false)) if forceLeftToBuild || forceRightToBuild { // Do not support specifying the build side. - p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(fmt.Sprintf("We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for %s, please check the hint", p.JoinType))) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack(fmt.Sprintf("We can't use the HASH_JOIN_BUILD or HASH_JOIN_PROBE hint for %s, please check the hint", p.JoinType))) forceLeftToBuild = false forceRightToBuild = false } @@ -445,7 +445,7 @@ func (p *LogicalJoin) getHashJoins(prop *property.PhysicalProperty) (joins []Phy if !forced && noHashJoin { return nil, false } else if forced && noHashJoin { - p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack( "Some HASH_JOIN and NO_HASH_JOIN hints conflict, NO_HASH_JOIN is ignored")) } return @@ -455,11 +455,11 @@ func (p *LogicalJoin) getHashJoin(prop *property.PhysicalProperty, innerIdx int, chReqProps := make([]*property.PhysicalProperty, 2) chReqProps[innerIdx] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus} chReqProps[1-innerIdx] = &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus} - if prop.ExpectedCnt < p.stats.RowCount { - expCntScale := prop.ExpectedCnt / p.stats.RowCount - chReqProps[1-innerIdx].ExpectedCnt = p.children[1-innerIdx].statsInfo().RowCount * expCntScale + if prop.ExpectedCnt < p.StatsInfo().RowCount { + expCntScale := prop.ExpectedCnt / p.StatsInfo().RowCount + chReqProps[1-innerIdx].ExpectedCnt = p.children[1-innerIdx].StatsInfo().RowCount * expCntScale } - hashJoin := NewPhysicalHashJoin(p, innerIdx, useOuterToBuild, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), chReqProps...) + hashJoin := NewPhysicalHashJoin(p, innerIdx, useOuterToBuild, p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), chReqProps...) hashJoin.SetSchema(p.schema) return hashJoin } @@ -498,9 +498,9 @@ func (p *LogicalJoin) constructIndexJoin( } chReqProps := make([]*property.PhysicalProperty, 2) chReqProps[outerIdx] = &property.PhysicalProperty{TaskTp: property.RootTaskType, ExpectedCnt: math.MaxFloat64, SortItems: prop.SortItems, CTEProducerStatus: prop.CTEProducerStatus} - if prop.ExpectedCnt < p.stats.RowCount { - expCntScale := prop.ExpectedCnt / p.stats.RowCount - chReqProps[outerIdx].ExpectedCnt = p.children[outerIdx].statsInfo().RowCount * expCntScale + if prop.ExpectedCnt < p.StatsInfo().RowCount { + expCntScale := prop.ExpectedCnt / p.StatsInfo().RowCount + chReqProps[outerIdx].ExpectedCnt = p.children[outerIdx].StatsInfo().RowCount * expCntScale } newInnerKeys := make([]*expression.Column, 0, len(innerJoinKeys)) newOuterKeys := make([]*expression.Column, 0, len(outerJoinKeys)) @@ -572,7 +572,7 @@ func (p *LogicalJoin) constructIndexJoin( CompareFilters: compareFilters, OuterHashKeys: outerHashKeys, InnerHashKeys: innerHashKeys, - }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) + }.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.SelectBlockOffset(), chReqProps...) if path != nil { join.IdxColLens = path.IdxColLens } @@ -659,8 +659,8 @@ func (p *LogicalJoin) constructIndexMergeJoin( if isOuterKeysPrefix && !prop.SortItems[i].Col.Equal(nil, join.OuterJoinKeys[keyOff2KeyOffOrderByIdx[i]]) { isOuterKeysPrefix = false } - compareFuncs = append(compareFuncs, expression.GetCmpFunction(p.ctx, join.OuterJoinKeys[i], join.InnerJoinKeys[i])) - outerCompareFuncs = append(outerCompareFuncs, expression.GetCmpFunction(p.ctx, join.OuterJoinKeys[i], join.OuterJoinKeys[i])) + compareFuncs = append(compareFuncs, expression.GetCmpFunction(p.SCtx(), join.OuterJoinKeys[i], join.InnerJoinKeys[i])) + outerCompareFuncs = append(outerCompareFuncs, expression.GetCmpFunction(p.SCtx(), join.OuterJoinKeys[i], join.OuterJoinKeys[i])) } // canKeepOuterOrder means whether the prop items are the prefix of the outer join keys. canKeepOuterOrder := len(prop.SortItems) <= len(join.OuterJoinKeys) @@ -680,7 +680,7 @@ func (p *LogicalJoin) constructIndexMergeJoin( CompareFuncs: compareFuncs, OuterCompareFuncs: outerCompareFuncs, Desc: !prop.IsSortItemEmpty() && prop.SortItems[0].Desc, - }.Init(p.ctx) + }.Init(p.SCtx()) indexMergeJoins = append(indexMergeJoins, indexMergeJoin) } } @@ -705,7 +705,7 @@ func (p *LogicalJoin) constructIndexHashJoin( // Prop is empty means that the parent operator does not need the // join operator to provide any promise of the output order. KeepOuterOrder: !prop.IsSortItemEmpty(), - }.Init(p.ctx) + }.Init(p.SCtx()) indexHashJoins = append(indexHashJoins, indexHashJoin) } return indexHashJoins @@ -737,8 +737,8 @@ func (p *LogicalJoin) getIndexJoinByOuterIdx(prop *property.PhysicalProperty, ou } var avgInnerRowCnt float64 - if outerChild.statsInfo().RowCount > 0 { - avgInnerRowCnt = p.equalCondOutCnt / outerChild.statsInfo().RowCount + if outerChild.StatsInfo().RowCount > 0 { + avgInnerRowCnt = p.equalCondOutCnt / outerChild.StatsInfo().RowCount } joins = p.buildIndexJoinInner2TableScan(prop, innerChildWrapper, innerJoinKeys, outerJoinKeys, outerIdx, avgInnerRowCnt) if joins != nil { @@ -773,7 +773,7 @@ func (p *LogicalJoin) extractIndexJoinInnerChildPattern(innerChild LogicalPlan) } } case *LogicalProjection: - if !p.ctx.GetSessionVars().EnableINLJoinInnerMultiPattern { + if !p.SCtx().GetSessionVars().EnableINLJoinInnerMultiPattern { return nil } // For now, we only allow proj with all Column expression can be the inner side of index join @@ -789,7 +789,7 @@ func (p *LogicalJoin) extractIndexJoinInnerChildPattern(innerChild LogicalPlan) } wrapper.ds = ds case *LogicalSelection: - if !p.ctx.GetSessionVars().EnableINLJoinInnerMultiPattern { + if !p.SCtx().GetSessionVars().EnableINLJoinInnerMultiPattern { return nil } wrapper.sel = child @@ -926,7 +926,7 @@ func (p *LogicalJoin) buildIndexJoinInner2TableScan( } joins = make([]PhysicalPlan, 0, 3) failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { - if val.(bool) && !p.ctx.GetSessionVars().InRestrictedSQL { + if val.(bool) && !p.SCtx().GetSessionVars().InRestrictedSQL { failpoint.Return(p.constructIndexHashJoin(prop, outerIdx, innerTask, nil, keyOff2IdxOff, path, lastColMng)) } }) @@ -963,7 +963,7 @@ func (p *LogicalJoin) buildIndexJoinInner2IndexScan( } innerTask := p.constructInnerIndexScanTask(wrapper, helper.chosenPath, helper.chosenRanges.Range(), helper.chosenRemained, innerJoinKeys, helper.idxOff2KeyOff, rangeInfo, false, false, avgInnerRowCnt, maxOneRow) failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { - if val.(bool) && !p.ctx.GetSessionVars().InRestrictedSQL { + if val.(bool) && !p.SCtx().GetSessionVars().InRestrictedSQL { failpoint.Return(p.constructIndexHashJoin(prop, outerIdx, innerTask, helper.chosenRanges, keyOff2IdxOff, helper.chosenPath, helper.lastColManager)) } }) @@ -1060,7 +1060,7 @@ func (p *LogicalJoin) constructInnerTableScanTask( isPartition: ds.isPartition, tblCols: ds.TblCols, tblColHists: ds.TblColHists, - }.Init(ds.ctx, ds.blockOffset) + }.Init(ds.SCtx(), ds.SelectBlockOffset()) ts.SetSchema(ds.schema.Clone()) if rowCount <= 0 { rowCount = float64(1) @@ -1069,7 +1069,7 @@ func (p *LogicalJoin) constructInnerTableScanTask( countAfterAccess := rowCount if len(ts.filterCondition) > 0 { var err error - selectivity, _, err = ds.tableStats.HistColl.Selectivity(ds.ctx, ts.filterCondition, ds.possibleAccessPaths) + selectivity, _, err = ds.tableStats.HistColl.Selectivity(ds.SCtx(), ts.filterCondition, ds.possibleAccessPaths) if err != nil || selectivity <= 0 { logutil.BgLogger().Debug("unexpected selectivity, use selection factor", zap.Float64("selectivity", selectivity), zap.String("table", ts.TableAsName.L)) selectivity = SelectionFactor @@ -1078,13 +1078,13 @@ func (p *LogicalJoin) constructInnerTableScanTask( // i.e, rowCount equals to `countAfterAccess * selectivity`. countAfterAccess = rowCount / selectivity } - ts.stats = &property.StatsInfo{ + ts.SetStats(&property.StatsInfo{ // TableScan as inner child of IndexJoin can return at most 1 tuple for each outer row. RowCount: math.Min(1.0, countAfterAccess), - StatsVersion: ds.stats.StatsVersion, + StatsVersion: ds.StatsInfo().StatsVersion, // NDV would not be used in cost computation of IndexJoin, set leave it as default nil. - } - usedStats := p.ctx.GetSessionVars().StmtCtx.GetUsedStatsInfo(false) + }) + usedStats := p.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(false) if usedStats != nil && usedStats[ts.physicalTableID] != nil { ts.usedStatsInfo = usedStats[ts.physicalTableID] } @@ -1101,16 +1101,16 @@ func (p *LogicalJoin) constructInnerTableScanTask( ColumnNames: ds.names, } ts.PartitionInfo = copTask.partitionInfo - selStats := ts.stats.Scale(selectivity) + selStats := ts.StatsInfo().Scale(selectivity) ts.addPushedDownSelection(copTask, selStats) - t := copTask.convertToRootTask(ds.ctx) + t := copTask.convertToRootTask(ds.SCtx()) reader := t.p t.p = p.constructInnerByWrapper(wrapper, reader) return t } func (p *LogicalJoin) constructInnerByWrapper(wrapper *indexJoinInnerChildWrapper, child PhysicalPlan) PhysicalPlan { - if !p.ctx.GetSessionVars().EnableINLJoinInnerMultiPattern { + if !p.SCtx().GetSessionVars().EnableINLJoinInnerMultiPattern { if wrapper.us != nil { return p.constructInnerUnionScan(wrapper.us, child) } @@ -1132,7 +1132,7 @@ func (*LogicalJoin) constructInnerSel(sel *LogicalSelection, child PhysicalPlan) } physicalSel := PhysicalSelection{ Conditions: sel.Conditions, - }.Init(sel.ctx, sel.stats, sel.blockOffset, nil) + }.Init(sel.SCtx(), sel.StatsInfo(), sel.SelectBlockOffset(), nil) physicalSel.SetChildren(child) return physicalSel } @@ -1145,7 +1145,7 @@ func (*LogicalJoin) constructInnerProj(proj *LogicalProjection, child PhysicalPl Exprs: proj.Exprs, CalculateNoDelay: proj.CalculateNoDelay, AvoidColumnEvaluator: proj.AvoidColumnEvaluator, - }.Init(proj.ctx, proj.stats, proj.blockOffset, nil) + }.Init(proj.SCtx(), proj.StatsInfo(), proj.SelectBlockOffset(), nil) physicalProj.SetChildren(child) return physicalProj } @@ -1154,12 +1154,12 @@ func (*LogicalJoin) constructInnerUnionScan(us *LogicalUnionScan, reader Physica if us == nil { return reader } - // Use `reader.stats` instead of `us.stats` because it should be more accurate. No need to specify + // Use `reader.StatsInfo()` instead of `us.StatsInfo()` because it should be more accurate. No need to specify // childrenReqProps now since we have got reader already. physicalUnionScan := PhysicalUnionScan{ Conditions: us.conditions, HandleCols: us.handleCols, - }.Init(us.ctx, reader.statsInfo(), us.blockOffset, nil) + }.Init(us.SCtx(), reader.StatsInfo(), us.SelectBlockOffset(), nil) physicalUnionScan.SetChildren(reader) return physicalUnionScan } @@ -1251,7 +1251,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( physicalTableID: ds.physicalTableID, tblColHists: ds.TblColHists, pkIsHandleCol: ds.getPKIsHandleCol(), - }.Init(ds.ctx, ds.blockOffset) + }.Init(ds.SCtx(), ds.SelectBlockOffset()) cop := &copTask{ indexPlan: is, tblColHists: ds.TblColHists, @@ -1275,7 +1275,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( physicalTableID: ds.physicalTableID, tblCols: ds.TblCols, tblColHists: ds.TblColHists, - }.Init(ds.ctx, ds.blockOffset) + }.Init(ds.SCtx(), ds.SelectBlockOffset()) ts.schema = is.dataSourceSchema.Clone() if ds.tableInfo.IsCommonHandle { commonHandle := ds.handleCols.(*CommonHandleCols) @@ -1290,8 +1290,8 @@ func (p *LogicalJoin) constructInnerIndexScanTask( // We set `StatsVersion` here and fill other fields in `(*copTask).finishIndexPlan`. Since `copTask.indexPlan` may // change before calling `(*copTask).finishIndexPlan`, we don't know the stats information of `ts` currently and on // the other hand, it may be hard to identify `StatsVersion` of `ts` in `(*copTask).finishIndexPlan`. - ts.stats = &property.StatsInfo{StatsVersion: ds.tableStats.StatsVersion} - usedStats := p.ctx.GetSessionVars().StmtCtx.GetUsedStatsInfo(false) + ts.SetStats(&property.StatsInfo{StatsVersion: ds.tableStats.StatsVersion}) + usedStats := p.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(false) if usedStats != nil && usedStats[ts.physicalTableID] != nil { ts.usedStatsInfo = usedStats[ts.physicalTableID] } @@ -1318,7 +1318,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( // the estimated row count of the IndexScan should be no larger than (total row count / NDV of join key columns). // We can calculate the lower bound of the NDV therefore we can get an upper bound of the row count here. rowCountUpperBound := -1.0 - fixControlOK := fixcontrol.GetBoolWithDefault(ds.ctx.GetSessionVars().GetOptimizerFixControlMap(), fixcontrol.Fix44855, false) + fixControlOK := fixcontrol.GetBoolWithDefault(ds.SCtx().GetSessionVars().GetOptimizerFixControlMap(), fixcontrol.Fix44855, false) if fixControlOK && ds.tableStats != nil { usedColIDs := make([]int64, 0) // We only consider columns in this index that (1) are used to probe as join key, @@ -1354,7 +1354,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( } // Assume equal conditions used by index join and other conditions are independent. if len(tblConds) > 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, tblConds, ds.possibleAccessPaths) + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), tblConds, ds.possibleAccessPaths) if err != nil || selectivity <= 0 { logutil.BgLogger().Debug("unexpected selectivity, use selection factor", zap.Float64("selectivity", selectivity), zap.String("table", ds.TableAsName.L)) selectivity = SelectionFactor @@ -1372,7 +1372,7 @@ func (p *LogicalJoin) constructInnerIndexScanTask( tmpPath.CountAfterAccess = cnt } if len(indexConds) > 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, indexConds, ds.possibleAccessPaths) + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), indexConds, ds.possibleAccessPaths) if err != nil || selectivity <= 0 { logutil.BgLogger().Debug("unexpected selectivity, use selection factor", zap.Float64("selectivity", selectivity), zap.String("table", ds.TableAsName.L)) selectivity = SelectionFactor @@ -1386,14 +1386,14 @@ func (p *LogicalJoin) constructInnerIndexScanTask( } tmpPath.CountAfterAccess = cnt } - is.stats = ds.tableStats.ScaleByExpectCnt(tmpPath.CountAfterAccess) - usedStats := ds.ctx.GetSessionVars().StmtCtx.GetUsedStatsInfo(false) + is.SetStats(ds.tableStats.ScaleByExpectCnt(tmpPath.CountAfterAccess)) + usedStats := ds.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(false) if usedStats != nil && usedStats[is.physicalTableID] != nil { is.usedStatsInfo = usedStats[is.physicalTableID] } finalStats := ds.tableStats.ScaleByExpectCnt(rowCount) is.addPushedDownSelection(cop, ds, tmpPath, finalStats) - t := cop.convertToRootTask(ds.ctx) + t := cop.convertToRootTask(ds.SCtx()) reader := t.p t.p = p.constructInnerByWrapper(wrapper, reader) return t @@ -1526,7 +1526,7 @@ func (ijHelper *indexJoinBuildHelper) resetContextForIndex(innerKeys []*expressi if ijHelper.curIdxOff2KeyOff[i] >= 0 { // Don't use the join columns if their collations are unmatched and the new collation is enabled. if collate.NewCollationEnabled() && types.IsString(idxCol.RetType.GetType()) && types.IsString(outerKeys[ijHelper.curIdxOff2KeyOff[i]].RetType.GetType()) { - et, err := expression.CheckAndDeriveCollationFromExprs(ijHelper.innerPlan.ctx, "equal", types.ETInt, idxCol, outerKeys[ijHelper.curIdxOff2KeyOff[i]]) + et, err := expression.CheckAndDeriveCollationFromExprs(ijHelper.innerPlan.SCtx(), "equal", types.ETInt, idxCol, outerKeys[ijHelper.curIdxOff2KeyOff[i]]) if err != nil { logutil.BgLogger().Error("Unexpected error happened during constructing index join", zap.Stack("stack")) } @@ -1549,7 +1549,7 @@ func (ijHelper *indexJoinBuildHelper) findUsefulEqAndInFilters(innerPlan *DataSo // Extract the eq/in functions of possible join key. // you can see the comment of ExtractEqAndInCondition to get the meaning of the second return value. usefulEqOrInFilters, remainedEqOrIn, remainingRangeCandidates, _, emptyRange = ranger.ExtractEqAndInCondition( - innerPlan.ctx, innerPlan.pushedDownConds, + innerPlan.SCtx(), innerPlan.pushedDownConds, ijHelper.curNotUsedIndexCols, ijHelper.curNotUsedColLens, ) @@ -1656,7 +1656,7 @@ func (mr *mutableIndexJoinRange) Rebuild() error { func (ijHelper *indexJoinBuildHelper) createMutableIndexJoinRange(relatedExprs []expression.Expression, ranges []*ranger.Range, path *util.AccessPath, innerKeys, outerKeys []*expression.Column) ranger.MutableRanges { // if the plan-cache is enabled and these ranges depend on some parameters, we have to rebuild these ranges after changing parameters - if expression.MaybeOverOptimized4PlanCache(ijHelper.join.ctx, relatedExprs) { + if expression.MaybeOverOptimized4PlanCache(ijHelper.join.SCtx(), relatedExprs) { // assume that path, innerKeys and outerKeys will not be modified in the follow-up process return &mutableIndexJoinRange{ ranges: ranges, @@ -1706,7 +1706,7 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath if lastColPos <= 0 { return false, nil } - rangeMaxSize := ijHelper.join.ctx.GetSessionVars().RangeMaxSize + rangeMaxSize := ijHelper.join.SCtx().GetSessionVars().RangeMaxSize if rebuildMode { // When rebuilding ranges for plan cache, we don't restrict range mem limit. rangeMaxSize = 0 @@ -1744,12 +1744,12 @@ func (ijHelper *indexJoinBuildHelper) analyzeLookUpFilters(path *util.AccessPath if matchedKeyCnt <= 0 { return false, nil } - colAccesses, colRemained := ranger.DetachCondsForColumn(ijHelper.join.ctx, rangeFilterCandidates, lastPossibleCol) + colAccesses, colRemained := ranger.DetachCondsForColumn(ijHelper.join.SCtx(), rangeFilterCandidates, lastPossibleCol) var nextColRange []*ranger.Range var err error if len(colAccesses) > 0 { var colRemained2 []expression.Expression - nextColRange, colAccesses, colRemained2, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.ctx, lastPossibleCol.RetType, path.IdxColLens[lastColPos], rangeMaxSize) + nextColRange, colAccesses, colRemained2, err = ranger.BuildColumnRange(colAccesses, ijHelper.join.SCtx(), lastPossibleCol.RetType, path.IdxColLens[lastColPos], rangeMaxSize) if err != nil { return false, err } @@ -1815,7 +1815,7 @@ func (ijHelper *indexJoinBuildHelper) updateBestChoice(ranges ranger.MutableRang return } var innerNDV float64 - if stats := ijHelper.innerPlan.statsInfo(); stats != nil && stats.StatsVersion != statistics.PseudoVersion { + if stats := ijHelper.innerPlan.StatsInfo(); stats != nil && stats.StatsVersion != statistics.PseudoVersion { innerNDV, _ = getColsNDVWithMatchedLen(path.IdxCols[:usedColsLen], ijHelper.innerPlan.Schema(), stats) } // We choose the index by the NDV of the used columns, the larger the better. @@ -1866,7 +1866,7 @@ func appendTailTemplateRange(originRanges ranger.Ranges, rangeMaxSize int64) (ra func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAndInFuncs []expression.Expression, nextColRange []*ranger.Range, haveExtraCol bool, rangeMaxSize int64) (res *templateRangeResult) { res = &templateRangeResult{} - ctx := ijHelper.join.ctx + ctx := ijHelper.join.SCtx() sc := ctx.GetSessionVars().StmtCtx defer func() { if sc.MemTracker != nil && res != nil && len(res.ranges) > 0 { @@ -1890,7 +1890,7 @@ func (ijHelper *indexJoinBuildHelper) buildTemplateRange(matchedKeyCnt int, eqAn i++ } else { exprs := []expression.Expression{eqAndInFuncs[j]} - oneColumnRan, _, remained, err := ranger.BuildColumnRange(exprs, ijHelper.join.ctx, ijHelper.curNotUsedIndexCols[j].RetType, ijHelper.curNotUsedColLens[j], rangeMaxSize) + oneColumnRan, _, remained, err := ranger.BuildColumnRange(exprs, ijHelper.join.SCtx(), ijHelper.curNotUsedIndexCols[j].RetType, ijHelper.curNotUsedColLens[j], rangeMaxSize) if err != nil { return &templateRangeResult{err: err} } @@ -2014,7 +2014,7 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ // Generate warning message to client. warning := ErrInternal.GenWithStack(errMsg) - p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) } }() @@ -2052,7 +2052,7 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ } switch { case len(forcedLeftOuterJoins) == 0 && !supportRightOuter: - return filterIndexJoinBySessionVars(p.ctx, allLeftOuterJoins), false + return filterIndexJoinBySessionVars(p.SCtx(), allLeftOuterJoins), false case len(forcedLeftOuterJoins) != 0 && (!supportRightOuter || (forceLeftOuter && !forceRightOuter)): return forcedLeftOuterJoins, true } @@ -2078,7 +2078,7 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ } switch { case len(forcedRightOuterJoins) == 0 && !supportLeftOuter: - return filterIndexJoinBySessionVars(p.ctx, allRightOuterJoins), false + return filterIndexJoinBySessionVars(p.SCtx(), allRightOuterJoins), false case len(forcedRightOuterJoins) != 0 && (!supportLeftOuter || (forceRightOuter && !forceLeftOuter)): return forcedRightOuterJoins, true } @@ -2090,20 +2090,20 @@ func (p *LogicalJoin) tryToGetIndexJoin(prop *property.PhysicalProperty) (indexJ if canForced { return append(forcedLeftOuterJoins, forcedRightOuterJoins...), true } - return filterIndexJoinBySessionVars(p.ctx, append(allLeftOuterJoins, allRightOuterJoins...)), false + return filterIndexJoinBySessionVars(p.SCtx(), append(allLeftOuterJoins, allRightOuterJoins...)), false } func checkChildFitBC(p Plan) bool { - if p.statsInfo().HistColl == nil { - return p.SCtx().GetSessionVars().BroadcastJoinThresholdCount == -1 || p.statsInfo().Count() < p.SCtx().GetSessionVars().BroadcastJoinThresholdCount + if p.StatsInfo().HistColl == nil { + return p.SCtx().GetSessionVars().BroadcastJoinThresholdCount == -1 || p.StatsInfo().Count() < p.SCtx().GetSessionVars().BroadcastJoinThresholdCount } - avg := p.statsInfo().HistColl.GetAvgRowSize(p.SCtx(), p.Schema().Columns, false, false) - sz := avg * float64(p.statsInfo().Count()) + avg := p.StatsInfo().HistColl.GetAvgRowSize(p.SCtx(), p.Schema().Columns, false, false) + sz := avg * float64(p.StatsInfo().Count()) return p.SCtx().GetSessionVars().BroadcastJoinThresholdSize == -1 || sz < float64(p.SCtx().GetSessionVars().BroadcastJoinThresholdSize) } func calcBroadcastExchangeSize(p Plan, mppStoreCnt int) (row float64, size float64, hasSize bool) { - s := p.statsInfo() + s := p.StatsInfo() row = float64(s.Count()) * float64(mppStoreCnt-1) if s.HistColl == nil { return row, 0, false @@ -2130,7 +2130,7 @@ func calcBroadcastExchangeSizeByChild(p1 Plan, p2 Plan, mppStoreCnt int) (row fl } func calcHashExchangeSize(p Plan, mppStoreCnt int) (row float64, sz float64, hasSize bool) { - s := p.statsInfo() + s := p.StatsInfo() row = float64(s.Count()) * float64(mppStoreCnt-1) / float64(mppStoreCnt) if s.HistColl == nil { return row, 0, false @@ -2180,15 +2180,15 @@ func isJoinChildFitMPPBCJ(p *LogicalJoin, childIndexToBC int, mppStoreCnt int) b // If we can use mpp broadcast join, that's our first choice. func (p *LogicalJoin) preferMppBCJ() bool { - if len(p.EqualConditions) == 0 && p.ctx.GetSessionVars().AllowCartesianBCJ == 2 { + if len(p.EqualConditions) == 0 && p.SCtx().GetSessionVars().AllowCartesianBCJ == 2 { return true } onlyCheckChild1 := p.JoinType == LeftOuterJoin || p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin onlyCheckChild0 := p.JoinType == RightOuterJoin - if p.ctx.GetSessionVars().PreferBCJByExchangeDataSize { - mppStoreCnt, err := p.ctx.GetMPPClient().GetMPPStoreCount() + if p.SCtx().GetSessionVars().PreferBCJByExchangeDataSize { + mppStoreCnt, err := p.SCtx().GetMPPClient().GetMPPStoreCount() // No need to exchange data if there is only ONE mpp store. But the behavior of optimizer is unexpected if use broadcast way forcibly, such as tpch q4. // TODO: always use broadcast way to exchange data if there is only ONE mpp store. @@ -2223,7 +2223,7 @@ func (p *LogicalJoin) preferMppBCJ() bool { // If the hint is not figured, we will pick all candidates. func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { failpoint.Inject("MockOnlyEnableIndexHashJoin", func(val failpoint.Value) { - if val.(bool) && !p.ctx.GetSessionVars().InRestrictedSQL { + if val.(bool) && !p.SCtx().GetSessionVars().InRestrictedSQL { indexJoins, _ := p.tryToGetIndexJoin(prop) failpoint.Return(indexJoins, true, nil) } @@ -2247,7 +2247,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } joins := make([]PhysicalPlan, 0, 8) canPushToTiFlash := p.canPushToCop(kv.TiFlash) - if p.ctx.GetSessionVars().IsMPPAllowed() && canPushToTiFlash { + if p.SCtx().GetSessionVars().IsMPPAllowed() && canPushToTiFlash { if (p.preferJoinType & preferShuffleJoin) > 0 { if shuffleJoins := p.tryToGetMppHashJoin(prop, false); len(shuffleJoins) > 0 { return shuffleJoins, true, nil @@ -2278,7 +2278,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P } if hasMppHints { warning := ErrInternal.GenWithStack(errMsg) - p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) } } if prop.IsFlashProp() { @@ -2287,7 +2287,7 @@ func (p *LogicalJoin) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P if !p.isNAAJ() { // naaj refuse merge join and index join. - mergeJoins := p.GetMergeJoin(prop, p.schema, p.Stats(), p.children[0].statsInfo(), p.children[1].statsInfo()) + mergeJoins := p.GetMergeJoin(prop, p.schema, p.StatsInfo(), p.children[0].StatsInfo(), p.children[1].StatsInfo()) if (p.preferJoinType&preferMergeJoin) > 0 && len(mergeJoins) > 0 { return mergeJoins, true, nil } @@ -2323,16 +2323,16 @@ func canExprsInJoinPushdown(p *LogicalJoin, storeType kv.StoreType) bool { } equalExprs = append(equalExprs, eqCondition) } - if !expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, equalExprs, p.ctx.GetClient(), storeType) { + if !expression.CanExprsPushDown(p.SCtx().GetSessionVars().StmtCtx, equalExprs, p.SCtx().GetClient(), storeType) { return false } - if !expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, p.LeftConditions, p.ctx.GetClient(), storeType) { + if !expression.CanExprsPushDown(p.SCtx().GetSessionVars().StmtCtx, p.LeftConditions, p.SCtx().GetClient(), storeType) { return false } - if !expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, p.RightConditions, p.ctx.GetClient(), storeType) { + if !expression.CanExprsPushDown(p.SCtx().GetSessionVars().StmtCtx, p.RightConditions, p.SCtx().GetClient(), storeType) { return false } - if !expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, p.OtherConditions, p.ctx.GetClient(), storeType) { + if !expression.CanExprsPushDown(p.SCtx().GetSessionVars().StmtCtx, p.OtherConditions, p.SCtx().GetClient(), storeType) { return false } return true @@ -2361,7 +2361,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because `Cartesian Product` is only supported by broadcast join, check value and documents of variables `tidb_broadcast_join_threshold_size` and `tidb_broadcast_join_threshold_count`.") return nil } - if p.ctx.GetSessionVars().AllowCartesianBCJ == 0 { + if p.SCtx().GetSessionVars().AllowCartesianBCJ == 0 { p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because `Cartesian Product` is only supported by broadcast join, check value and documents of variable `tidb_opt_broadcast_cartesian_join`.") return nil } @@ -2399,14 +2399,14 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC forceLeftToBuild := ((p.preferJoinType & preferLeftAsHJBuild) > 0) || ((p.preferJoinType & preferRightAsHJProbe) > 0) forceRightToBuild := ((p.preferJoinType & preferRightAsHJBuild) > 0) || ((p.preferJoinType & preferLeftAsHJProbe) > 0) if forceLeftToBuild && forceRightToBuild { - p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints")) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints are conflicts, please check the hints")) forceLeftToBuild = false forceRightToBuild = false } preferredBuildIndex := 0 fixedBuildSide := false // Used to indicate whether the build side for the MPP join is fixed or not. if p.JoinType == InnerJoin { - if p.children[0].statsInfo().Count() > p.children[1].statsInfo().Count() { + if p.children[0].StatsInfo().Count() > p.children[1].StatsInfo().Count() { preferredBuildIndex = 1 } } else if p.JoinType.IsSemiJoin() { @@ -2415,7 +2415,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC preferredBuildIndex = 1 // MPPOuterJoinFixedBuildSide default value is false // use MPPOuterJoinFixedBuildSide here as a way to disable using left table as build side - if !p.ctx.GetSessionVars().MPPOuterJoinFixedBuildSide && p.children[1].statsInfo().Count() > p.children[0].statsInfo().Count() { + if !p.SCtx().GetSessionVars().MPPOuterJoinFixedBuildSide && p.children[1].StatsInfo().Count() > p.children[0].StatsInfo().Count() { preferredBuildIndex = 0 } } else { @@ -2429,15 +2429,15 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC // 1. it is a broadcast join(for broadcast join, it makes sense to use the broadcast side as the build side) // 2. or session variable MPPOuterJoinFixedBuildSide is set to true // 3. or nullAware/cross joins - if useBCJ || p.isNAAJ() || len(p.EqualConditions) == 0 || p.ctx.GetSessionVars().MPPOuterJoinFixedBuildSide { - if !p.ctx.GetSessionVars().MPPOuterJoinFixedBuildSide { + if useBCJ || p.isNAAJ() || len(p.EqualConditions) == 0 || p.SCtx().GetSessionVars().MPPOuterJoinFixedBuildSide { + if !p.SCtx().GetSessionVars().MPPOuterJoinFixedBuildSide { // The hint has higher priority than variable. fixedBuildSide = true } if p.JoinType == LeftOuterJoin { preferredBuildIndex = 1 } - } else if p.children[0].statsInfo().Count() > p.children[1].statsInfo().Count() { + } else if p.children[0].StatsInfo().Count() > p.children[1].StatsInfo().Count() { preferredBuildIndex = 1 } } @@ -2447,7 +2447,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC if !match { if fixedBuildSide { // A warning will be generated if the build side is fixed, but we attempt to change it using the hint. - p.ctx.GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints")) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(ErrInternal.GenWithStack("Some HASH_JOIN_BUILD and HASH_JOIN_PROBE hints cannot be utilized for MPP joins, please check the hints")) } else { // The HASH_JOIN_BUILD OR HASH_JOIN_PROBE hints can take effective. preferredBuildIndex = 1 - preferredBuildIndex @@ -2457,7 +2457,7 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC // set preferredBuildIndex for test failpoint.Inject("mockPreferredBuildIndex", func(val failpoint.Value) { - if !p.ctx.GetSessionVars().InRestrictedSQL { + if !p.SCtx().GetSessionVars().InRestrictedSQL { preferredBuildIndex = val.(int) } }) @@ -2467,9 +2467,9 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC if useBCJ { childrenProps[preferredBuildIndex] = &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.BroadcastType, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} expCnt := math.MaxFloat64 - if prop.ExpectedCnt < p.stats.RowCount { - expCntScale := prop.ExpectedCnt / p.stats.RowCount - expCnt = p.children[1-preferredBuildIndex].statsInfo().RowCount * expCntScale + if prop.ExpectedCnt < p.StatsInfo().RowCount { + expCntScale := prop.ExpectedCnt / p.StatsInfo().RowCount + expCnt = p.children[1-preferredBuildIndex].StatsInfo().RowCount * expCntScale } if prop.MPPPartitionTp == property.HashType { lPartitionKeys, rPartitionKeys := p.GetPotentialPartitionKeys() @@ -2517,13 +2517,13 @@ func (p *LogicalJoin) tryToGetMppHashJoin(prop *property.PhysicalProperty, useBC } join := PhysicalHashJoin{ basePhysicalJoin: baseJoin, - Concurrency: uint(p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor), + Concurrency: uint(p.SCtx().GetSessionVars().CopTiFlashConcurrencyFactor), EqualConditions: p.EqualConditions, NAEqualConditions: p.NAEQConditions, storeTp: kv.TiFlash, mppShuffleJoin: !useBCJ, // Mpp Join has quite heavy cost. Even limit might not suspend it in time, so we don't scale the count. - }.Init(p.ctx, p.stats, p.blockOffset, childrenProps...) + }.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), childrenProps...) join.SetSchema(p.schema) return []PhysicalPlan{join} } @@ -2580,7 +2580,7 @@ func (p *LogicalExpand) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ GroupingSets: p.rollupGroupingSets, LevelExprs: p.LevelExprs, ExtraGroupingColNames: p.ExtraGroupingColNames, - }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, mppProp) + }.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.SelectBlockOffset(), mppProp) expand.SetSchema(p.Schema()) return []PhysicalPlan{expand}, true, nil } @@ -2614,7 +2614,7 @@ func (p *LogicalProjection) exhaustPhysicalPlans(prop *property.PhysicalProperty Exprs: p.Exprs, CalculateNoDelay: p.CalculateNoDelay, AvoidColumnEvaluator: p.AvoidColumnEvaluator, - }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, newProp) + }.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.SelectBlockOffset(), newProp) proj.SetSchema(p.schema) ret = append(ret, proj) } @@ -2627,7 +2627,7 @@ func pushLimitOrTopNForcibly(p LogicalPlan) bool { switch lp := p.(type) { case *LogicalTopN: preferPushDown = &lp.limitHints.preferLimitToCop - meetThreshold = lp.Count+lp.Offset <= uint64(lp.ctx.GetSessionVars().LimitPushDownThreshold) + meetThreshold = lp.Count+lp.Offset <= uint64(lp.SCtx().GetSessionVars().LimitPushDownThreshold) case *LogicalLimit: preferPushDown = &lp.limitHints.preferLimitToCop meetThreshold = true // always push Limit down in this case since it has no side effect @@ -2655,7 +2655,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl if !pushLimitOrTopNForcibly(lt) { allTaskTypes = append(allTaskTypes, property.RootTaskType) } - if lt.ctx.GetSessionVars().IsMPPAllowed() { + if lt.SCtx().GetSessionVars().IsMPPAllowed() { allTaskTypes = append(allTaskTypes, property.MppTaskType) } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) @@ -2666,7 +2666,7 @@ func (lt *LogicalTopN) getPhysTopN(prop *property.PhysicalProperty) []PhysicalPl PartitionBy: lt.PartitionBy, Count: lt.Count, Offset: lt.Offset, - }.Init(lt.ctx, lt.stats, lt.blockOffset, resultProp) + }.Init(lt.SCtx(), lt.StatsInfo(), lt.SelectBlockOffset(), resultProp) ret = append(ret, topN) } return ret @@ -2689,7 +2689,7 @@ func (lt *LogicalTopN) getPhysLimits(prop *property.PhysicalProperty) []Physical Count: lt.Count, Offset: lt.Offset, PartitionBy: lt.GetPartitionBy(), - }.Init(lt.ctx, lt.stats, lt.blockOffset, resultProp) + }.Init(lt.SCtx(), lt.StatsInfo(), lt.SelectBlockOffset(), resultProp) limit.SetSchema(lt.Schema()) ret = append(ret, limit) } @@ -2729,7 +2729,7 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ return nil, true, nil } if !prop.IsSortItemEmpty() && la.SCtx().GetSessionVars().EnableParallelApply { - la.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Parallel Apply rejects the possible order properties of its outer child currently")) + la.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.Errorf("Parallel Apply rejects the possible order properties of its outer child currently")) return nil, true, nil } disableAggPushDownToCop(la.children[0]) @@ -2739,15 +2739,15 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ columns = append(columns, &colColumn.Column) } cacheHitRatio := 0.0 - if la.stats.RowCount != 0 { - ndv, _ := getColsNDVWithMatchedLen(columns, la.schema, la.stats) + if la.StatsInfo().RowCount != 0 { + ndv, _ := getColsNDVWithMatchedLen(columns, la.schema, la.StatsInfo()) // for example, if there are 100 rows and the number of distinct values of these correlated columns // are 70, then we can assume 30 rows can hit the cache so the cache hit ratio is 1 - (70/100) = 0.3 - cacheHitRatio = 1 - (ndv / la.stats.RowCount) + cacheHitRatio = 1 - (ndv / la.StatsInfo().RowCount) } var canUseCache bool - if cacheHitRatio > 0.1 && la.ctx.GetSessionVars().MemQuotaApplyCache > 0 { + if cacheHitRatio > 0.1 && la.SCtx().GetSessionVars().MemQuotaApplyCache > 0 { canUseCache = true } else { canUseCache = false @@ -2757,9 +2757,9 @@ func (la *LogicalApply) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([ PhysicalHashJoin: *join, OuterSchema: la.CorCols, CanUseCache: canUseCache, - }.Init(la.ctx, - la.stats.ScaleByExpectCnt(prop.ExpectedCnt), - la.blockOffset, + }.Init(la.SCtx(), + la.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), + la.SelectBlockOffset(), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, SortItems: prop.SortItems, CTEProducerStatus: prop.CTEProducerStatus}, &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, CTEProducerStatus: prop.CTEProducerStatus}) apply.SetSchema(la.schema) @@ -2815,12 +2815,12 @@ func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []P return nil } if lw.Frame != nil && lw.Frame.Type == ast.Ranges { - if _, err := expression.ExpressionsToPBList(lw.SCtx().GetSessionVars().StmtCtx, lw.Frame.Start.CalcFuncs, lw.ctx.GetClient()); err != nil { + if _, err := expression.ExpressionsToPBList(lw.SCtx().GetSessionVars().StmtCtx, lw.Frame.Start.CalcFuncs, lw.SCtx().GetClient()); err != nil { lw.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced( "MPP mode may be blocked because window function frame can't be pushed down, because " + err.Error()) return nil } - if _, err := expression.ExpressionsToPBList(lw.SCtx().GetSessionVars().StmtCtx, lw.Frame.End.CalcFuncs, lw.ctx.GetClient()); err != nil { + if _, err := expression.ExpressionsToPBList(lw.SCtx().GetSessionVars().StmtCtx, lw.Frame.End.CalcFuncs, lw.SCtx().GetClient()); err != nil { lw.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced( "MPP mode may be blocked because window function frame can't be pushed down, because " + err.Error()) return nil @@ -2873,7 +2873,7 @@ func (lw *LogicalWindow) tryToGetMppWindows(prop *property.PhysicalProperty) []P OrderBy: lw.OrderBy, Frame: lw.Frame, storeTp: kv.TiFlash, - }.Init(lw.ctx, lw.stats.ScaleByExpectCnt(prop.ExpectedCnt), lw.blockOffset, childProperty) + }.Init(lw.SCtx(), lw.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), lw.SelectBlockOffset(), childProperty) window.SetSchema(lw.Schema()) return []PhysicalPlan{window} @@ -2883,7 +2883,7 @@ func (lw *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ( windows := make([]PhysicalPlan, 0, 2) canPushToTiFlash := lw.canPushToCop(kv.TiFlash) - if lw.ctx.GetSessionVars().IsMPPAllowed() && canPushToTiFlash { + if lw.SCtx().GetSessionVars().IsMPPAllowed() && canPushToTiFlash { mppWindows := lw.tryToGetMppWindows(prop) windows = append(windows, mppWindows...) } @@ -2904,7 +2904,7 @@ func (lw *LogicalWindow) exhaustPhysicalPlans(prop *property.PhysicalProperty) ( PartitionBy: lw.PartitionBy, OrderBy: lw.OrderBy, Frame: lw.Frame, - }.Init(lw.ctx, lw.stats.ScaleByExpectCnt(prop.ExpectedCnt), lw.blockOffset, childProperty) + }.Init(lw.SCtx(), lw.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), lw.SelectBlockOffset(), childProperty) window.SetSchema(lw.Schema()) windows = append(windows, window) @@ -3014,7 +3014,7 @@ func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalPrope allTaskTypes := prop.GetAllPossibleChildTaskTypes() enforcedAggs := make([]PhysicalPlan, 0, len(allTaskTypes)) childProp := &property.PhysicalProperty{ - ExpectedCnt: math.Max(prop.ExpectedCnt*la.inputCount/la.stats.RowCount, prop.ExpectedCnt), + ExpectedCnt: math.Max(prop.ExpectedCnt*la.inputCount/la.StatsInfo().RowCount, prop.ExpectedCnt), CanAddEnforcer: true, SortItems: property.SortItemsFromCols(la.GetGroupByCols(), desc), } @@ -3025,7 +3025,7 @@ func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalPrope if la.HasDistinct() { // TODO: remove AllowDistinctAggPushDown after the cost estimation of distinct pushdown is implemented. // If AllowDistinctAggPushDown is set to true, we should not consider RootTask. - if !la.canPushToCop(kv.TiKV) || !la.ctx.GetSessionVars().AllowDistinctAggPushDown { + if !la.canPushToCop(kv.TiKV) || !la.SCtx().GetSessionVars().AllowDistinctAggPushDown { taskTypes = []property.TaskType{property.RootTaskType} } } else if !la.aggHints.preferAggToCop { @@ -3044,7 +3044,7 @@ func (la *LogicalAggregation) getEnforcedStreamAggs(prop *property.PhysicalPrope agg := basePhysicalAgg{ GroupByItems: newGbyItems, AggFuncs: newAggFuncs, - }.initForStream(la.ctx, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), la.blockOffset, copiedChildProperty) + }.initForStream(la.SCtx(), la.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), la.SelectBlockOffset(), copiedChildProperty) agg.SetSchema(la.schema.Clone()) enforcedAggs = append(enforcedAggs, agg) } @@ -3088,7 +3088,7 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P allTaskTypes := prop.GetAllPossibleChildTaskTypes() streamAggs := make([]PhysicalPlan, 0, len(la.possibleProperties)*(len(allTaskTypes)-1)+len(allTaskTypes)) childProp := &property.PhysicalProperty{ - ExpectedCnt: math.Max(prop.ExpectedCnt*la.inputCount/la.stats.RowCount, prop.ExpectedCnt), + ExpectedCnt: math.Max(prop.ExpectedCnt*la.inputCount/la.StatsInfo().RowCount, prop.ExpectedCnt), } for _, possibleChildProperty := range la.possibleProperties { @@ -3102,7 +3102,7 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P if la.HasDistinct() { // TODO: remove AllowDistinctAggPushDown after the cost estimation of distinct pushdown is implemented. // If AllowDistinctAggPushDown is set to true, we should not consider RootTask. - if !la.ctx.GetSessionVars().AllowDistinctAggPushDown || !la.canPushToCop(kv.TiKV) { + if !la.SCtx().GetSessionVars().AllowDistinctAggPushDown || !la.canPushToCop(kv.TiKV) { // if variable doesn't allow DistinctAggPushDown, just produce root task type. // if variable does allow DistinctAggPushDown, but OP itself can't be pushed down to tikv, just produce root task type. taskTypes = []property.TaskType{property.RootTaskType} @@ -3128,7 +3128,7 @@ func (la *LogicalAggregation) getStreamAggs(prop *property.PhysicalProperty) []P agg := basePhysicalAgg{ GroupByItems: newGbyItems, AggFuncs: newAggFuncs, - }.initForStream(la.ctx, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), la.blockOffset, copiedChildProperty) + }.initForStream(la.SCtx(), la.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), la.SelectBlockOffset(), copiedChildProperty) agg.SetSchema(la.schema.Clone()) streamAggs = append(streamAggs, agg) } @@ -3158,14 +3158,14 @@ func (la *LogicalAggregation) checkCanPushDownToMPP() bool { } if hasUnsupportedDistinct { warnErr := errors.New("Aggregation can not be pushed to storage layer in mpp mode because it contains agg function with distinct") - if la.ctx.GetSessionVars().StmtCtx.InExplainStmt { - la.ctx.GetSessionVars().StmtCtx.AppendWarning(warnErr) + if la.SCtx().GetSessionVars().StmtCtx.InExplainStmt { + la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warnErr) } else { - la.ctx.GetSessionVars().StmtCtx.AppendExtraWarning(warnErr) + la.SCtx().GetSessionVars().StmtCtx.AppendExtraWarning(warnErr) } return false } - return CheckAggCanPushCop(la.ctx, la.AggFuncs, la.GroupByItems, kv.TiFlash) + return CheckAggCanPushCop(la.SCtx(), la.AggFuncs, la.GroupByItems, kv.TiFlash) } func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalProperty) (hashAggs []PhysicalPlan) { @@ -3209,9 +3209,9 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // --> after fill null in Expand --> and this shown two rows should be shuffled to the same node (the underlying partition is not satisfied yet) // <1,1> in node A <1,null,gid=1> in node A // <1,2> in node B <1,null,gid=1> in node B - if len(partitionCols) != 0 && !la.ctx.GetSessionVars().EnableSkewDistinctAgg { + if len(partitionCols) != 0 && !la.SCtx().GetSessionVars().EnableSkewDistinctAgg { childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.HashType, MPPPartitionCols: partitionCols, CanAddEnforcer: true, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} - agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) + agg := NewPhysicalHashAgg(la, la.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = Mpp1Phase hashAggs = append(hashAggs, agg) @@ -3225,7 +3225,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // 2-phase agg // no partition property down,record partition cols inside agg itself, enforce shuffler latter. childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, MPPPartitionTp: property.AnyType, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} - agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) + agg := NewPhysicalHashAgg(la, la.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = Mpp2Phase agg.MppPartitionCols = partitionCols @@ -3234,7 +3234,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert // agg runs on TiDB with a partial agg on TiFlash if possible if prop.TaskTp == property.RootTaskType { childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} - agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) + agg := NewPhysicalHashAgg(la, la.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) agg.MppRunMode = MppTiDB hashAggs = append(hashAggs, agg) @@ -3242,7 +3242,7 @@ func (la *LogicalAggregation) tryToGetMppHashAggs(prop *property.PhysicalPropert } else if !hasFinalAgg { // TODO: support scalar agg in MPP, merge the final result to one node childProp := &property.PhysicalProperty{TaskTp: property.MppTaskType, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus} - agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) + agg := NewPhysicalHashAgg(la, la.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), childProp) agg.SetSchema(la.schema.Clone()) if la.HasDistinct() || la.HasOrderBy() { // mpp scalar mode means the data will be pass through to only one tiFlash node at last. @@ -3292,10 +3292,10 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy hashAggs := make([]PhysicalPlan, 0, len(prop.GetAllPossibleChildTaskTypes())) taskTypes := []property.TaskType{property.CopSingleReadTaskType, property.CopMultiReadTaskType} canPushDownToTiFlash := la.canPushToCop(kv.TiFlash) - canPushDownToMPP := canPushDownToTiFlash && la.ctx.GetSessionVars().IsMPPAllowed() && la.checkCanPushDownToMPP() + canPushDownToMPP := canPushDownToTiFlash && la.SCtx().GetSessionVars().IsMPPAllowed() && la.checkCanPushDownToMPP() if la.HasDistinct() { // TODO: remove after the cost estimation of distinct pushdown is implemented. - if !la.ctx.GetSessionVars().AllowDistinctAggPushDown || !la.canPushToCop(kv.TiKV) { + if !la.SCtx().GetSessionVars().AllowDistinctAggPushDown || !la.canPushToCop(kv.TiKV) { // if variable doesn't allow DistinctAggPushDown, just produce root task type. // if variable does allow DistinctAggPushDown, but OP itself can't be pushed down to tikv, just produce root task type. taskTypes = []property.TaskType{property.RootTaskType} @@ -3321,7 +3321,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy } if hasMppHints { warning := ErrInternal.GenWithStack(errMsg) - la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) } } if prop.IsFlashProp() { @@ -3335,7 +3335,7 @@ func (la *LogicalAggregation) getHashAggs(prop *property.PhysicalProperty) []Phy hashAggs = append(hashAggs, mppAggs...) } } else { - agg := NewPhysicalHashAgg(la, la.stats.ScaleByExpectCnt(prop.ExpectedCnt), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, TaskTp: taskTp, CTEProducerStatus: prop.CTEProducerStatus}) + agg := NewPhysicalHashAgg(la, la.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64, TaskTp: taskTp, CTEProducerStatus: prop.CTEProducerStatus}) agg.SetSchema(la.schema.Clone()) hashAggs = append(hashAggs, agg) } @@ -3351,7 +3351,7 @@ func (la *LogicalAggregation) ResetHintIfConflicted() (preferHash bool, preferSt if preferHash && preferStream { errMsg := "Optimizer aggregation hints are conflicted" warning := ErrInternal.GenWithStack(errMsg) - la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) la.aggHints.preferAggType = 0 preferHash, preferStream = false, false } @@ -3363,7 +3363,7 @@ func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProper if !la.canPushToCop(kv.TiKV) { errMsg := "Optimizer Hint AGG_TO_COP is inapplicable" warning := ErrInternal.GenWithStack(errMsg) - la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) la.aggHints.preferAggToCop = false } } @@ -3385,7 +3385,7 @@ func (la *LogicalAggregation) exhaustPhysicalPlans(prop *property.PhysicalProper if streamAggs == nil && preferStream && !prop.IsSortItemEmpty() { errMsg := "Optimizer Hint STREAM_AGG is inapplicable" warning := ErrInternal.GenWithStack(errMsg) - la.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + la.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) } return aggs, !(preferStream || preferHash), nil @@ -3408,7 +3408,7 @@ func (p *LogicalSelection) exhaustPhysicalPlans(prop *property.PhysicalProperty) for _, newProp := range newProps { sel := PhysicalSelection{ Conditions: p.Conditions, - }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, newProp) + }.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.SelectBlockOffset(), newProp) ret = append(ret, sel) } return ret, true, nil @@ -3434,7 +3434,7 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] if !pushLimitOrTopNForcibly(p) { allTaskTypes = append(allTaskTypes, property.RootTaskType) } - if p.canPushToCop(kv.TiFlash) && p.ctx.GetSessionVars().IsMPPAllowed() { + if p.canPushToCop(kv.TiFlash) && p.SCtx().GetSessionVars().IsMPPAllowed() { allTaskTypes = append(allTaskTypes, property.MppTaskType) } ret := make([]PhysicalPlan, 0, len(allTaskTypes)) @@ -3444,7 +3444,7 @@ func (p *LogicalLimit) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] Offset: p.Offset, Count: p.Count, PartitionBy: p.GetPartitionBy(), - }.Init(p.ctx, p.stats, p.blockOffset, resultProp) + }.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), resultProp) limit.SetSchema(p.Schema()) ret = append(ret, limit) } @@ -3462,7 +3462,7 @@ func (p *LogicalLock) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]P Lock: p.Lock, TblID2Handle: p.tblID2Handle, TblID2PhysTblIDCol: p.tblID2PhysTblIDCol, - }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), childProp) + }.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), childProp) return []PhysicalPlan{lock}, true, nil } @@ -3475,7 +3475,7 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) if prop.TaskTp == property.MppTaskType && prop.MPPPartitionTp != property.AnyType { return nil, true, nil } - canUseMpp := p.ctx.GetSessionVars().IsMPPAllowed() && p.canPushToCopImpl(kv.TiFlash, true) + canUseMpp := p.SCtx().GetSessionVars().IsMPPAllowed() && p.canPushToCopImpl(kv.TiFlash, true) chReqProps := make([]*property.PhysicalProperty, 0, len(p.children)) for range p.children { if canUseMpp && prop.TaskTp == property.MppTaskType { @@ -3491,7 +3491,7 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) } ua := PhysicalUnionAll{ mpp: canUseMpp && prop.TaskTp == property.MppTaskType, - }.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) + }.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.SelectBlockOffset(), chReqProps...) ua.SetSchema(p.Schema()) if canUseMpp && prop.TaskTp == property.RootTaskType { chReqProps = make([]*property.PhysicalProperty, 0, len(p.children)) @@ -3503,7 +3503,7 @@ func (p *LogicalUnionAll) exhaustPhysicalPlans(prop *property.PhysicalProperty) CTEProducerStatus: prop.CTEProducerStatus, }) } - mppUA := PhysicalUnionAll{mpp: true}.Init(p.ctx, p.stats.ScaleByExpectCnt(prop.ExpectedCnt), p.blockOffset, chReqProps...) + mppUA := PhysicalUnionAll{mpp: true}.Init(p.SCtx(), p.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), p.SelectBlockOffset(), chReqProps...) mppUA.SetSchema(p.Schema()) return []PhysicalPlan{ua, mppUA}, true, nil } @@ -3516,13 +3516,13 @@ func (p *LogicalPartitionUnionAll) exhaustPhysicalPlans(prop *property.PhysicalP return nil, false, err } for _, ua := range uas { - ua.(*PhysicalUnionAll).tp = plancodec.TypePartitionUnion + ua.(*PhysicalUnionAll).SetTP(plancodec.TypePartitionUnion) } return uas, flagHint, nil } func (ls *LogicalSort) getPhysicalSort(prop *property.PhysicalProperty) *PhysicalSort { - ps := PhysicalSort{ByItems: ls.ByItems}.Init(ls.ctx, ls.stats.ScaleByExpectCnt(prop.ExpectedCnt), ls.blockOffset, &property.PhysicalProperty{TaskTp: prop.TaskTp, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus}) + ps := PhysicalSort{ByItems: ls.ByItems}.Init(ls.SCtx(), ls.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), ls.SelectBlockOffset(), &property.PhysicalProperty{TaskTp: prop.TaskTp, ExpectedCnt: math.MaxFloat64, RejectSort: true, CTEProducerStatus: prop.CTEProducerStatus}) return ps } @@ -3534,7 +3534,7 @@ func (ls *LogicalSort) getNominalSort(reqProp *property.PhysicalProperty) *Nomin prop.RejectSort = true prop.ExpectedCnt = reqProp.ExpectedCnt ps := NominalSort{OnlyColumn: onlyColumn, ByItems: ls.ByItems}.Init( - ls.ctx, ls.stats.ScaleByExpectCnt(prop.ExpectedCnt), ls.blockOffset, prop) + ls.SCtx(), ls.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), ls.SelectBlockOffset(), prop) return ps } @@ -3554,7 +3554,7 @@ func (ls *LogicalSort) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([] newProp := prop.CloneEssentialFields() newProp.RejectSort = true ps := NominalSort{OnlyColumn: true, ByItems: ls.ByItems}.Init( - ls.ctx, ls.stats.ScaleByExpectCnt(prop.ExpectedCnt), ls.blockOffset, newProp) + ls.SCtx(), ls.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), ls.SelectBlockOffset(), newProp) return []PhysicalPlan{ps}, true, nil } } @@ -3566,16 +3566,16 @@ func (p *LogicalMaxOneRow) exhaustPhysicalPlans(prop *property.PhysicalProperty) p.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because operator `MaxOneRow` is not supported now.") return nil, true, nil } - mor := PhysicalMaxOneRow{}.Init(p.ctx, p.stats, p.blockOffset, &property.PhysicalProperty{ExpectedCnt: 2, CTEProducerStatus: prop.CTEProducerStatus}) + mor := PhysicalMaxOneRow{}.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), &property.PhysicalProperty{ExpectedCnt: 2, CTEProducerStatus: prop.CTEProducerStatus}) return []PhysicalPlan{mor}, true, nil } func (p *LogicalCTE) exhaustPhysicalPlans(prop *property.PhysicalProperty) ([]PhysicalPlan, bool, error) { - pcte := PhysicalCTE{CTE: p.cte}.Init(p.ctx, p.stats) + pcte := PhysicalCTE{CTE: p.cte}.Init(p.SCtx(), p.StatsInfo()) if prop.IsFlashProp() { pcte.storageSender = PhysicalExchangeSender{ ExchangeType: tipb.ExchangeType_Broadcast, - }.Init(p.ctx, p.stats) + }.Init(p.SCtx(), p.StatsInfo()) } pcte.SetSchema(p.schema) pcte.childrenReqProps = []*property.PhysicalProperty{prop.CloneEssentialFields()} @@ -3608,7 +3608,7 @@ func (p *LogicalSequence) exhaustPhysicalPlans(prop *property.PhysicalProperty) childReqs = append(childReqs, propChoice[0].CloneEssentialFields()) } childReqs = append(childReqs, propChoice[1]) - seq := PhysicalSequence{}.Init(p.ctx, p.stats, p.blockOffset, childReqs...) + seq := PhysicalSequence{}.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), childReqs...) seq.SetSchema(p.children[len(p.children)-1].Schema()) seqs = append(seqs, seq) } diff --git a/planner/core/exhaust_physical_plans_test.go b/planner/core/exhaust_physical_plans_test.go index 2de0fc834e6a9..ae3564dd27755 100644 --- a/planner/core/exhaust_physical_plans_test.go +++ b/planner/core/exhaust_physical_plans_test.go @@ -108,7 +108,7 @@ func prepareForAnalyzeLookUpFilters() *indexJoinContext { DBName: model.NewCIStr("test"), }) dataSourceNode.schema = dsSchema - dataSourceNode.stats = &property.StatsInfo{StatsVersion: statistics.PseudoVersion} + dataSourceNode.SetStats(&property.StatsInfo{StatsVersion: statistics.PseudoVersion}) path := &util.AccessPath{ IdxCols: append(make([]*expression.Column, 0, 5), dsSchema.Columns...), IdxColLens: []int{types.UnspecifiedLength, types.UnspecifiedLength, 2, types.UnspecifiedLength, 2}, @@ -179,7 +179,7 @@ type indexJoinTestCase struct { } func testAnalyzeLookUpFilters(t *testing.T, testCtx *indexJoinContext, testCase *indexJoinTestCase, msgAndArgs ...interface{}) *indexJoinBuildHelper { - ctx := testCtx.dataSourceNode.ctx + ctx := testCtx.dataSourceNode.SCtx() ctx.GetSessionVars().RangeMaxSize = testCase.rangeMaxSize dataSourceNode := testCtx.dataSourceNode joinNode := testCtx.joinNode @@ -343,7 +343,7 @@ func checkRangeFallbackAndReset(t *testing.T, ctx sessionctx.Context, expectedRa func TestRangeFallbackForAnalyzeLookUpFilters(t *testing.T) { ijCtx := prepareForAnalyzeLookUpFilters() - ctx := ijCtx.dataSourceNode.ctx + ctx := ijCtx.dataSourceNode.SCtx() dsSchema := ijCtx.dataSourceNode.schema type testOutput struct { diff --git a/planner/core/explain.go b/planner/core/explain.go index e52d29d0ac7d7..3a1ad8a0041c7 100644 --- a/planner/core/explain.go +++ b/planner/core/explain.go @@ -48,10 +48,10 @@ func (p *PhysicalLock) ExplainInfo() string { // ExplainID overrides the ExplainID in order to match different range. func (p *PhysicalIndexScan) ExplainID() fmt.Stringer { return stringutil.MemoizeStr(func() string { - if p.ctx != nil && p.ctx.GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { + if p.SCtx() != nil && p.SCtx().GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { return p.TP() } - return p.TP() + "_" + strconv.Itoa(p.id) + return p.TP() + "_" + strconv.Itoa(p.ID()) }) } @@ -120,7 +120,7 @@ func (p *PhysicalIndexScan) OperatorInfo(normalized bool) string { buffer.WriteString(", ") buffer.WriteString(str) } - } else if p.stats.StatsVersion == statistics.PseudoVersion { + } else if p.StatsInfo().StatsVersion == statistics.PseudoVersion { // This branch is not needed in fact, we add this to prevent test result changes under planner/cascades/ buffer.WriteString(", stats:pseudo") } @@ -152,10 +152,10 @@ func (p *PhysicalIndexScan) isFullScan() bool { // ExplainID overrides the ExplainID in order to match different range. func (p *PhysicalTableScan) ExplainID() fmt.Stringer { return stringutil.MemoizeStr(func() string { - if p.ctx != nil && p.ctx.GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { + if p.SCtx() != nil && p.SCtx().GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { return p.TP() } - return p.TP() + "_" + strconv.Itoa(p.id) + return p.TP() + "_" + strconv.Itoa(p.ID()) }) } @@ -213,7 +213,7 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { } } } - if p.ctx.GetSessionVars().EnableLateMaterialization && len(p.filterCondition) > 0 && p.StoreType == kv.TiFlash { + if p.SCtx().GetSessionVars().EnableLateMaterialization && len(p.filterCondition) > 0 && p.StoreType == kv.TiFlash { buffer.WriteString("pushed down filter:") if len(p.lateMaterializationFilterCondition) > 0 { if normalized { @@ -238,12 +238,12 @@ func (p *PhysicalTableScan) OperatorInfo(normalized bool) string { buffer.WriteString(", ") buffer.WriteString(str) } - } else if p.stats.StatsVersion == statistics.PseudoVersion { + } else if p.StatsInfo().StatsVersion == statistics.PseudoVersion { // This branch is not needed in fact, we add this to prevent test result changes under planner/cascades/ buffer.WriteString(", stats:pseudo") } } - if p.StoreType == kv.TiFlash && p.Table.GetPartitionInfo() != nil && p.IsMPPOrBatchCop && p.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { + if p.StoreType == kv.TiFlash && p.Table.GetPartitionInfo() != nil && p.IsMPPOrBatchCop && p.SCtx().GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { buffer.WriteString(", PartitionTableScan:true") } if len(p.runtimeFilterList) > 0 { @@ -290,7 +290,7 @@ func (p *PhysicalTableReader) ExplainInfo() string { tablePlanInfo := "data:" + p.tablePlan.ExplainID().String() if p.ReadReqType == MPP { - return fmt.Sprintf("MppVersion: %d, %s", p.ctx.GetSessionVars().ChooseMppVersion(), tablePlanInfo) + return fmt.Sprintf("MppVersion: %d, %s", p.SCtx().GetSessionVars().ChooseMppVersion(), tablePlanInfo) } return tablePlanInfo diff --git a/planner/core/find_best_task.go b/planner/core/find_best_task.go index 1540fd739bcf8..d2e6730b8cccf 100644 --- a/planner/core/find_best_task.go +++ b/planner/core/find_best_task.go @@ -147,7 +147,7 @@ func (p *LogicalTableDual) findBestTask(prop *property.PhysicalProperty, planCou } dual := PhysicalTableDual{ RowCount: p.RowCount, - }.Init(p.ctx, p.stats, p.blockOffset) + }.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset()) dual.SetSchema(p.schema) planCounter.Dec(1) opt.appendCandidate(p, dual, prop) @@ -158,7 +158,7 @@ func (p *LogicalShow) findBestTask(prop *property.PhysicalProperty, planCounter if !prop.IsSortItemEmpty() || planCounter.Empty() { return invalidTask, 0, nil } - pShow := PhysicalShow{ShowContents: p.ShowContents, Extractor: p.Extractor}.Init(p.ctx) + pShow := PhysicalShow{ShowContents: p.ShowContents, Extractor: p.Extractor}.Init(p.SCtx()) pShow.SetSchema(p.schema) planCounter.Dec(1) return &rootTask{p: pShow}, 1, nil @@ -168,7 +168,7 @@ func (p *LogicalShowDDLJobs) findBestTask(prop *property.PhysicalProperty, planC if !prop.IsSortItemEmpty() || planCounter.Empty() { return invalidTask, 0, nil } - pShow := PhysicalShowDDLJobs{JobNumber: p.JobNumber}.Init(p.ctx) + pShow := PhysicalShowDDLJobs{JobNumber: p.JobNumber}.Init(p.SCtx()) pShow.SetSchema(p.schema) planCounter.Dec(1) return &rootTask{p: pShow}, 1, nil @@ -252,7 +252,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl for _, pp := range physicalPlans { timeStampNow := p.GetLogicalTS4TaskMap() - savedPlanID := p.ctx.GetSessionVars().PlanID.Load() + savedPlanID := p.SCtx().GetSessionVars().PlanID.Load() childTasks, curCntPlan, childCnts, err = iteration(pp, childTasks, childCnts, prop, opt) @@ -267,7 +267,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl // If the target plan can be found in this physicalPlan(pp), rebuild childTasks to build the corresponding combination. if planCounter.IsForce() && int64(*planCounter) <= curCntPlan { - p.ctx.GetSessionVars().PlanID.Store(savedPlanID) + p.SCtx().GetSessionVars().PlanID.Store(savedPlanID) curCntPlan = int64(*planCounter) err := p.rebuildChildTasks(&childTasks, pp, childCnts, int64(*planCounter), timeStampNow, opt) if err != nil { @@ -284,18 +284,18 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl // An optimal task could not satisfy the property, so it should be converted here. if _, ok := curTask.(*rootTask); !ok && prop.TaskTp == property.RootTaskType { - curTask = curTask.convertToRootTask(p.ctx) + curTask = curTask.convertToRootTask(p.SCtx()) } // Enforce curTask property if addEnforcer { - curTask = enforceProperty(prop, curTask, p.basePlan.ctx) + curTask = enforceProperty(prop, curTask, p.Plan.SCtx()) } // Optimize by shuffle executor to running in parallel manner. if _, isMpp := curTask.(*mppTask); !isMpp && prop.IsSortItemEmpty() { // Currently, we do not regard shuffled plan as a new plan. - curTask = optimizeByShuffle(curTask, p.basePlan.ctx) + curTask = optimizeByShuffle(curTask, p.Plan.SCtx()) } cntPlan += curCntPlan @@ -307,7 +307,7 @@ func (p *baseLogicalPlan) enumeratePhysicalPlans4Task(physicalPlans []PhysicalPl } opt.appendCandidate(p, curTask.plan(), prop) // Get the most efficient one. - if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask, opt); err != nil { + if curIsBetter, err := compareTaskCost(p.SCtx(), curTask, bestTask, opt); err != nil { return nil, 0, err } else if curIsBetter { bestTask = curTask @@ -609,7 +609,7 @@ func (p *baseLogicalPlan) findBestTask(prop *property.PhysicalProperty, planCoun goto END } opt.appendCandidate(p, curTask.plan(), prop) - if curIsBetter, err := compareTaskCost(p.ctx, curTask, bestTask, opt); err != nil { + if curIsBetter, err := compareTaskCost(p.SCtx(), curTask, bestTask, opt); err != nil { return nil, 0, err } else if curIsBetter { bestTask = curTask @@ -650,7 +650,7 @@ func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, planCoun } if prop.CanAddEnforcer { *prop = *oldProp - t = enforceProperty(prop, t, p.basePlan.ctx) + t = enforceProperty(prop, t, p.Plan.SCtx()) prop.CanAddEnforcer = true } }() @@ -664,7 +664,7 @@ func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, planCoun Columns: p.Columns, Extractor: p.Extractor, QueryTimeRange: p.QueryTimeRange, - }.Init(p.ctx, p.stats, p.blockOffset) + }.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset()) memTable.SetSchema(p.schema) planCounter.Dec(1) opt.appendCandidate(p, memTable, prop) @@ -675,12 +675,12 @@ func (p *LogicalMemTable) findBestTask(prop *property.PhysicalProperty, planCoun func (ds *DataSource) tryToGetDualTask() (task, error) { for _, cond := range ds.pushedDownConds { if con, ok := cond.(*expression.Constant); ok && con.DeferredExpr == nil && con.ParamMarker == nil { - result, _, err := expression.EvalBool(ds.ctx, []expression.Expression{cond}, chunk.Row{}) + result, _, err := expression.EvalBool(ds.SCtx(), []expression.Expression{cond}, chunk.Row{}) if err != nil { return nil, err } if !result { - dual := PhysicalTableDual{}.Init(ds.ctx, ds.stats, ds.blockOffset) + dual := PhysicalTableDual{}.Init(ds.SCtx(), ds.StatsInfo(), ds.SelectBlockOffset()) dual.SetSchema(ds.schema) return &rootTask{ p: dual, @@ -866,7 +866,7 @@ func (ds *DataSource) skylinePruning(prop *property.PhysicalProperty) []*candida } } - if ds.ctx.GetSessionVars().GetAllowPreferRangeScan() && len(candidates) > 1 { + if ds.SCtx().GetSessionVars().GetAllowPreferRangeScan() && len(candidates) > 1 { // If a candidate path is TiFlash-path or forced-path, we just keep them. For other candidate paths, if there exists // any range scan path, we remove full scan paths and keep range scan paths. preferredPaths := make([]*candidatePath, 0, len(candidates)) @@ -961,7 +961,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter planCounter.Dec(1) return nil, 1, nil } - if ds.isForUpdateRead && ds.ctx.GetSessionVars().TxnCtx.IsExplicit { + if ds.isForUpdateRead && ds.SCtx().GetSessionVars().TxnCtx.IsExplicit { hasPointGetPath := false for _, path := range ds.possibleAccessPaths { if ds.isPointGetPath(path) { @@ -970,7 +970,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter } } tblName := ds.tableInfo.Name - ds.possibleAccessPaths, err = filterPathByIsolationRead(ds.ctx, ds.possibleAccessPaths, tblName, ds.DBName) + ds.possibleAccessPaths, err = filterPathByIsolationRead(ds.SCtx(), ds.possibleAccessPaths, tblName, ds.DBName) if err != nil { return nil, 1, err } @@ -1020,14 +1020,14 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter } if prop.CanAddEnforcer { *prop = *oldProp - t = enforceProperty(prop, t, ds.basePlan.ctx) + t = enforceProperty(prop, t, ds.Plan.SCtx()) prop.CanAddEnforcer = true } ds.storeTask(prop, t) if ds.SampleInfo != nil && !t.invalid() { if _, ok := t.plan().(*PhysicalTableSample); !ok { warning := expression.ErrInvalidTableSample.GenWithStackByArgs("plan not supported") - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + ds.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) } } }() @@ -1047,10 +1047,10 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter defer func() { if err == nil && t != nil && !t.invalid() && pruningInfo != "" { warnErr := errors.New(pruningInfo) - if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain { - ds.ctx.GetSessionVars().StmtCtx.AppendNote(warnErr) + if ds.SCtx().GetSessionVars().StmtCtx.InVerboseExplain { + ds.SCtx().GetSessionVars().StmtCtx.AppendNote(warnErr) } else { - ds.ctx.GetSessionVars().StmtCtx.AppendExtraNote(warnErr) + ds.SCtx().GetSessionVars().StmtCtx.AppendExtraNote(warnErr) } } }() @@ -1069,7 +1069,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter } appendCandidate(ds, idxMergeTask, prop, opt) - curIsBetter, err := compareTaskCost(ds.ctx, idxMergeTask, t, opt) + curIsBetter, err := compareTaskCost(ds.SCtx(), idxMergeTask, t, opt) if err != nil { return nil, 0, err } @@ -1084,10 +1084,10 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter // if we already know the range of the scan is empty, just return a TableDual if len(path.Ranges) == 0 { // We should uncache the tableDual plan. - if expression.MaybeOverOptimized4PlanCache(ds.ctx, path.AccessConds) { - ds.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("get a TableDual plan")) + if expression.MaybeOverOptimized4PlanCache(ds.SCtx(), path.AccessConds) { + ds.SCtx().GetSessionVars().StmtCtx.SetSkipPlanCache(errors.Errorf("get a TableDual plan")) } - dual := PhysicalTableDual{}.Init(ds.ctx, ds.stats, ds.blockOffset) + dual := PhysicalTableDual{}.Init(ds.SCtx(), ds.StatsInfo(), ds.SelectBlockOffset()) dual.SetSchema(ds.schema) cntPlan++ planCounter.Dec(1) @@ -1123,7 +1123,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter // decided by the current implementation of `BatchPointGetExec::initialize()`, specifically, // the `getPhysID()` function. Once we optimize that part, we can come back and enable // BatchPointGet plan for more cases. - hashPartColName = getHashOrKeyPartitionColumnName(ds.ctx, tblInfo) + hashPartColName = getHashOrKeyPartitionColumnName(ds.SCtx(), tblInfo) if hashPartColName == nil { canConvertPointGet = false } @@ -1142,7 +1142,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter if canConvertPointGet { allRangeIsPoint := true for _, ran := range path.Ranges { - if !ran.IsPointNonNullable(ds.ctx) { + if !ran.IsPointNonNullable(ds.SCtx()) { // unique indexes can have duplicated NULL rows so we cannot use PointGet if there is NULL allRangeIsPoint = false break @@ -1158,8 +1158,8 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter // Batch/PointGet plans may be over-optimized, like `a>=1(?) and a<=1(?)` --> `a=1` --> PointGet(a=1). // For safety, prevent these plans from the plan cache here. - if !pointGetTask.invalid() && expression.MaybeOverOptimized4PlanCache(ds.ctx, candidate.path.AccessConds) && !isSafePointGetPath4PlanCache(ds.ctx, candidate.path) { - ds.ctx.GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("Batch/PointGet plans may be over-optimized")) + if !pointGetTask.invalid() && expression.MaybeOverOptimized4PlanCache(ds.SCtx(), candidate.path.AccessConds) && !isSafePointGetPath4PlanCache(ds.SCtx(), candidate.path) { + ds.SCtx().GetSessionVars().StmtCtx.SetSkipPlanCache(errors.New("Batch/PointGet plans may be over-optimized")) } appendCandidate(ds, pointGetTask, prop, opt) @@ -1167,7 +1167,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter cntPlan++ planCounter.Dec(1) } - curIsBetter, cerr := compareTaskCost(ds.ctx, pointGetTask, t, opt) + curIsBetter, cerr := compareTaskCost(ds.SCtx(), pointGetTask, t, opt) if cerr != nil { return nil, 0, cerr } @@ -1201,7 +1201,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter planCounter.Dec(1) } appendCandidate(ds, tblTask, prop, opt) - curIsBetter, err := compareTaskCost(ds.ctx, tblTask, t, opt) + curIsBetter, err := compareTaskCost(ds.SCtx(), tblTask, t, opt) if err != nil { return nil, 0, err } @@ -1226,7 +1226,7 @@ func (ds *DataSource) findBestTask(prop *property.PhysicalProperty, planCounter planCounter.Dec(1) } appendCandidate(ds, idxTask, prop, opt) - curIsBetter, err := compareTaskCost(ds.ctx, idxTask, t, opt) + curIsBetter, err := compareTaskCost(ds.SCtx(), idxTask, t, opt) if err != nil { return nil, 0, err } @@ -1255,7 +1255,7 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c tblColHists: ds.TblColHists, } cop.partitionInfo = PartitionInfo{ - PruningConds: pushDownNot(ds.ctx, ds.allConds), + PruningConds: pushDownNot(ds.SCtx(), ds.allConds), PartitionNames: ds.partitionNames, Columns: ds.TblCols, ColumnNames: ds.names, @@ -1270,8 +1270,8 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c scans = append(scans, scan) } totalRowCount := path.CountAfterAccess - if prop.ExpectedCnt < ds.stats.RowCount { - totalRowCount *= prop.ExpectedCnt / ds.stats.RowCount + if prop.ExpectedCnt < ds.StatsInfo().RowCount { + totalRowCount *= prop.ExpectedCnt / ds.StatsInfo().RowCount } ts, remainingFilters, err := ds.buildIndexMergeTableScan(prop, path.TableFilters, totalRowCount) if err != nil { @@ -1290,7 +1290,7 @@ func (ds *DataSource) convertToIndexMergeScan(prop *property.PhysicalProperty, c } if prop.TaskTp == property.RootTaskType { cop.indexPlanFinished = true - task = cop.convertToRootTask(ds.ctx) + task = cop.convertToRootTask(ds.SCtx()) } else { task = cop } @@ -1306,13 +1306,13 @@ func (ds *DataSource) convertToPartialIndexScan(prop *property.PhysicalProperty, if path.CountAfterAccess > 0 { selectivity = path.CountAfterIndex / path.CountAfterAccess } - rowCount := is.stats.RowCount * selectivity + rowCount := is.StatsInfo().RowCount * selectivity stats := &property.StatsInfo{RowCount: rowCount} stats.StatsVersion = ds.statisticTable.Version if ds.statisticTable.Pseudo { stats.StatsVersion = statistics.PseudoVersion } - indexPlan := PhysicalSelection{Conditions: indexConds}.Init(is.ctx, stats, ds.blockOffset) + indexPlan := PhysicalSelection{Conditions: indexConds}.Init(is.SCtx(), stats, ds.SelectBlockOffset()) indexPlan.SetChildren(is) return indexPlan } @@ -1342,12 +1342,12 @@ func (ds *DataSource) convertToPartialTableScan(prop *property.PhysicalProperty, } ts.filterCondition = newFilterConds if len(ts.filterCondition) > 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, ts.filterCondition, nil) + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), ts.filterCondition, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor } - tablePlan = PhysicalSelection{Conditions: ts.filterCondition}.Init(ts.ctx, ts.stats.ScaleByExpectCnt(selectivity*rowCount), ds.blockOffset) + tablePlan = PhysicalSelection{Conditions: ts.filterCondition}.Init(ts.SCtx(), ts.StatsInfo().ScaleByExpectCnt(selectivity*rowCount), ds.SelectBlockOffset()) tablePlan.SetChildren(ts) return tablePlan } @@ -1397,7 +1397,7 @@ func setIndexMergeTableScanHandleCols(ds *DataSource, ts *PhysicalTableScan) (er // Filters that cannot be pushed to TiKV are also returned, and an extra Selection above IndexMergeReader will be constructed later. func (ds *DataSource) buildIndexMergeTableScan(_ *property.PhysicalProperty, tableFilters []expression.Expression, totalRowCount float64) (PhysicalPlan, []expression.Expression, error) { - sessVars := ds.ctx.GetSessionVars() + sessVars := ds.SCtx().GetSessionVars() ts := PhysicalTableScan{ Table: ds.tableInfo, Columns: ds.Columns, @@ -1408,7 +1408,7 @@ func (ds *DataSource) buildIndexMergeTableScan(_ *property.PhysicalProperty, tab HandleCols: ds.handleCols, tblCols: ds.TblCols, tblColHists: ds.TblColHists, - }.Init(ds.ctx, ds.blockOffset) + }.Init(ds.SCtx(), ds.SelectBlockOffset()) ts.SetSchema(ds.schema.Clone()) err := setIndexMergeTableScanHandleCols(ds, ts) if err != nil { @@ -1421,26 +1421,26 @@ func (ds *DataSource) buildIndexMergeTableScan(_ *property.PhysicalProperty, tab } } } - ts.stats = ds.tableStats.ScaleByExpectCnt(totalRowCount) - usedStats := ds.ctx.GetSessionVars().StmtCtx.GetUsedStatsInfo(false) + ts.SetStats(ds.tableStats.ScaleByExpectCnt(totalRowCount)) + usedStats := ds.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(false) if usedStats != nil && usedStats[ts.physicalTableID] != nil { ts.usedStatsInfo = usedStats[ts.physicalTableID] } if ds.statisticTable.Pseudo { - ts.stats.StatsVersion = statistics.PseudoVersion + ts.StatsInfo().StatsVersion = statistics.PseudoVersion } if len(tableFilters) > 0 { - pushedFilters, remainingFilters := extractFiltersForIndexMerge(sessVars.StmtCtx, ds.ctx.GetClient(), tableFilters) + pushedFilters, remainingFilters := extractFiltersForIndexMerge(sessVars.StmtCtx, ds.SCtx().GetClient(), tableFilters) pushedFilters1, remainingFilters1 := SplitSelCondsWithVirtualColumn(pushedFilters) pushedFilters = pushedFilters1 remainingFilters = append(remainingFilters, remainingFilters1...) if len(pushedFilters) != 0 { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, pushedFilters, nil) + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), pushedFilters, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor } - sel := PhysicalSelection{Conditions: pushedFilters}.Init(ts.ctx, ts.stats.ScaleByExpectCnt(selectivity*totalRowCount), ts.blockOffset) + sel := PhysicalSelection{Conditions: pushedFilters}.Init(ts.SCtx(), ts.StatsInfo().ScaleByExpectCnt(selectivity*totalRowCount), ts.SelectBlockOffset()) sel.SetChildren(ts) return sel, remainingFilters, nil } @@ -1534,7 +1534,7 @@ func (ds *DataSource) isIndexCoveringCondition(condition expression.Expression, } func (ds *DataSource) isSingleScan(indexColumns []*expression.Column, idxColLens []int) bool { - if !ds.ctx.GetSessionVars().OptPrefixIndexSingleScan || ds.colsRequiringFullLen == nil { + if !ds.SCtx().GetSessionVars().OptPrefixIndexSingleScan || ds.colsRequiringFullLen == nil { // ds.colsRequiringFullLen is set at (*DataSource).PruneColumns. In some cases we don't reach (*DataSource).PruneColumns // and ds.colsRequiringFullLen is nil, so we fall back to ds.isIndexCoveringColumns(ds.schema.Columns, indexColumns, idxColLens). return ds.isIndexCoveringColumns(ds.schema.Columns, indexColumns, idxColLens) @@ -1600,7 +1600,7 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, expectCnt: uint64(prop.ExpectedCnt), } cop.partitionInfo = PartitionInfo{ - PruningConds: pushDownNot(ds.ctx, ds.allConds), + PruningConds: pushDownNot(ds.SCtx(), ds.allConds), PartitionNames: ds.partitionNames, Columns: ds.TblCols, ColumnNames: ds.names, @@ -1616,13 +1616,13 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, physicalTableID: ds.physicalTableID, tblCols: ds.TblCols, tblColHists: ds.TblColHists, - }.Init(ds.ctx, is.blockOffset) + }.Init(ds.SCtx(), is.SelectBlockOffset()) ts.SetSchema(ds.schema.Clone()) // We set `StatsVersion` here and fill other fields in `(*copTask).finishIndexPlan`. Since `copTask.indexPlan` may // change before calling `(*copTask).finishIndexPlan`, we don't know the stats information of `ts` currently and on // the other hand, it may be hard to identify `StatsVersion` of `ts` in `(*copTask).finishIndexPlan`. - ts.stats = &property.StatsInfo{StatsVersion: ds.tableStats.StatsVersion} - usedStats := ds.ctx.GetSessionVars().StmtCtx.GetUsedStatsInfo(false) + ts.SetStats(&property.StatsInfo{StatsVersion: ds.tableStats.StatsVersion}) + usedStats := ds.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(false) if usedStats != nil && usedStats[ts.physicalTableID] != nil { ts.usedStatsInfo = usedStats[ts.physicalTableID] } @@ -1659,14 +1659,14 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, }) } cop.indexPlan.(*PhysicalIndexScan).ByItems = byItems - if cop.tablePlan != nil && ds.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { + if cop.tablePlan != nil && ds.SCtx().GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { if !is.Index.Global { - is.Columns, is.schema, _ = AddExtraPhysTblIDColumn(is.ctx, is.Columns, is.Schema()) + is.Columns, is.schema, _ = AddExtraPhysTblIDColumn(is.SCtx(), is.Columns, is.Schema()) } var succ bool // global index for tableScan with keepOrder also need PhysicalTblID ts := cop.tablePlan.(*PhysicalTableScan) - ts.Columns, ts.schema, succ = AddExtraPhysTblIDColumn(ts.ctx, ts.Columns, ts.Schema()) + ts.Columns, ts.schema, succ = AddExtraPhysTblIDColumn(ts.SCtx(), ts.Columns, ts.Schema()) cop.needExtraProj = cop.needExtraProj || succ } } @@ -1676,10 +1676,10 @@ func (ds *DataSource) convertToIndexScan(prop *property.PhysicalProperty, } // prop.IsSortItemEmpty() would always return true when coming to here, // so we can just use prop.ExpectedCnt as parameter of addPushedDownSelection. - finalStats := ds.stats.ScaleByExpectCnt(prop.ExpectedCnt) + finalStats := ds.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt) is.addPushedDownSelection(cop, ds, path, finalStats) if prop.TaskTp == property.RootTaskType { - task = task.convertToRootTask(ds.ctx) + task = task.convertToRootTask(ds.SCtx()) } else if _, ok := task.(*rootTask); ok { return invalidTask, nil } @@ -1699,7 +1699,7 @@ func (is *PhysicalIndexScan) getScanRowSize() float64 { } else { scanCols = is.schema.Columns } - return is.tblColHists.GetIndexAvgRowSize(is.ctx, scanCols, is.Index.Unique) + return is.tblColHists.GetIndexAvgRowSize(is.SCtx(), scanCols, is.Index.Unique) } // initSchema is used to set the schema of PhysicalIndexScan. Before calling this, @@ -1722,7 +1722,7 @@ func (is *PhysicalIndexScan) initSchema(idxExprCols []*expression.Column, isDoub indexCols = append(indexCols, &expression.Column{ ID: is.Table.Columns[is.Index.Columns[i].Offset].ID, RetType: &is.Table.Columns[is.Index.Columns[i].Offset].FieldType, - UniqueID: is.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: is.SCtx().GetSessionVars().AllocPlanColumnID(), }) } } @@ -1753,7 +1753,7 @@ func (is *PhysicalIndexScan) initSchema(idxExprCols []*expression.Column, isDoub indexCols = append(indexCols, &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), ID: model.ExtraHandleID, - UniqueID: is.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: is.SCtx().GetSessionVars().AllocPlanColumnID(), }) } } @@ -1762,7 +1762,7 @@ func (is *PhysicalIndexScan) initSchema(idxExprCols []*expression.Column, isDoub indexCols = append(indexCols, &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), ID: model.ExtraPidColID, - UniqueID: is.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: is.SCtx().GetSessionVars().AllocPlanColumnID(), }) } } @@ -1772,7 +1772,7 @@ func (is *PhysicalIndexScan) initSchema(idxExprCols []*expression.Column, isDoub indexCols = append(indexCols, &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), ID: model.ExtraPhysTblID, - UniqueID: is.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: is.SCtx().GetSessionVars().AllocPlanColumnID(), }) } @@ -1785,10 +1785,10 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSou tableConds, copTask.rootTaskConds = SplitSelCondsWithVirtualColumn(tableConds) var newRootConds []expression.Expression - indexConds, newRootConds = expression.PushDownExprs(is.ctx.GetSessionVars().StmtCtx, indexConds, is.ctx.GetClient(), kv.TiKV) + indexConds, newRootConds = expression.PushDownExprs(is.SCtx().GetSessionVars().StmtCtx, indexConds, is.SCtx().GetClient(), kv.TiKV) copTask.rootTaskConds = append(copTask.rootTaskConds, newRootConds...) - tableConds, newRootConds = expression.PushDownExprs(is.ctx.GetSessionVars().StmtCtx, tableConds, is.ctx.GetClient(), kv.TiKV) + tableConds, newRootConds = expression.PushDownExprs(is.SCtx().GetSessionVars().StmtCtx, tableConds, is.SCtx().GetClient(), kv.TiKV) copTask.rootTaskConds = append(copTask.rootTaskConds, newRootConds...) if indexConds != nil { @@ -1796,22 +1796,22 @@ func (is *PhysicalIndexScan) addPushedDownSelection(copTask *copTask, p *DataSou if path.CountAfterAccess > 0 { selectivity = path.CountAfterIndex / path.CountAfterAccess } - count := is.stats.RowCount * selectivity + count := is.StatsInfo().RowCount * selectivity stats := p.tableStats.ScaleByExpectCnt(count) - indexSel := PhysicalSelection{Conditions: indexConds}.Init(is.ctx, stats, is.blockOffset) + indexSel := PhysicalSelection{Conditions: indexConds}.Init(is.SCtx(), stats, is.SelectBlockOffset()) indexSel.SetChildren(is) copTask.indexPlan = indexSel } if len(tableConds) > 0 { copTask.finishIndexPlan() - tableSel := PhysicalSelection{Conditions: tableConds}.Init(is.ctx, finalStats, is.blockOffset) + tableSel := PhysicalSelection{Conditions: tableConds}.Init(is.SCtx(), finalStats, is.SelectBlockOffset()) if len(copTask.rootTaskConds) != 0 { - selectivity, _, err := copTask.tblColHists.Selectivity(is.ctx, tableConds, nil) + selectivity, _, err := copTask.tblColHists.Selectivity(is.SCtx(), tableConds, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor } - tableSel.stats = copTask.plan().statsInfo().Scale(selectivity) + tableSel.SetStats(copTask.plan().StatsInfo().Scale(selectivity)) } tableSel.SetChildren(copTask.tablePlan) copTask.tablePlan = tableSel @@ -1864,7 +1864,7 @@ func (ds *DataSource) splitIndexFilterConditions(conditions []expression.Express var indexConditions, tableConditions []expression.Expression for _, cond := range conditions { var covered bool - if ds.ctx.GetSessionVars().OptPrefixIndexSingleScan { + if ds.SCtx().GetSessionVars().OptPrefixIndexSingleScan { covered = ds.isIndexCoveringCondition(cond, indexColumns, idxColLens) } else { covered = ds.isIndexCoveringColumns(expression.ExtractColumns(cond), indexColumns, idxColLens) @@ -1975,10 +1975,10 @@ func convertRangeFromExpectedCnt(ranges []*ranger.Range, rangeCounts []float64, // if order of column `a` is strictly correlated with column `pk`, the row count of table scan should be: // `1 + row_count(a < 1 or a is null)` func (ds *DataSource) crossEstimateTableRowCount(path *util.AccessPath, expectedCnt float64, desc bool) (float64, bool, float64) { - if ds.statisticTable.Pseudo || len(path.TableFilters) == 0 || !ds.ctx.GetSessionVars().EnableCorrelationAdjustment { + if ds.statisticTable.Pseudo || len(path.TableFilters) == 0 || !ds.SCtx().GetSessionVars().EnableCorrelationAdjustment { return 0, false, 0 } - col, corr := getMostCorrCol4Handle(path.TableFilters, ds.statisticTable, ds.ctx.GetSessionVars().CorrelationThreshold) + col, corr := getMostCorrCol4Handle(path.TableFilters, ds.statisticTable, ds.SCtx().GetSessionVars().CorrelationThreshold) return ds.crossEstimateRowCount(path, path.TableFilters, col, corr, expectedCnt, desc) } @@ -1993,20 +1993,20 @@ func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, conds []expre if corr < 0 { desc = !desc } - accessConds, remained := ranger.DetachCondsForColumn(ds.ctx, conds, col) + accessConds, remained := ranger.DetachCondsForColumn(ds.SCtx(), conds, col) if len(accessConds) == 0 { return 0, false, corr } - ranges, accessConds, _, err := ranger.BuildColumnRange(accessConds, ds.ctx, col.RetType, types.UnspecifiedLength, ds.ctx.GetSessionVars().RangeMaxSize) + ranges, accessConds, _, err := ranger.BuildColumnRange(accessConds, ds.SCtx(), col.RetType, types.UnspecifiedLength, ds.SCtx().GetSessionVars().RangeMaxSize) if len(ranges) == 0 || len(accessConds) == 0 || err != nil { return 0, err == nil, corr } idxID := int64(-1) - idxIDs, idxExists := ds.stats.HistColl.ColID2IdxIDs[colID] + idxIDs, idxExists := ds.StatsInfo().HistColl.ColID2IdxIDs[colID] if idxExists && len(idxIDs) > 0 { idxID = idxIDs[0] } - rangeCounts, ok := getColumnRangeCounts(ds.ctx, colID, ranges, ds.tableStats.HistColl, idxID) + rangeCounts, ok := getColumnRangeCounts(ds.SCtx(), colID, ranges, ds.tableStats.HistColl, idxID) if !ok { return 0, false, corr } @@ -2016,9 +2016,9 @@ func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, conds []expre } var rangeCount float64 if idxExists { - rangeCount, err = ds.tableStats.HistColl.GetRowCountByIndexRanges(ds.ctx, idxID, convertedRanges) + rangeCount, err = ds.tableStats.HistColl.GetRowCountByIndexRanges(ds.SCtx(), idxID, convertedRanges) } else { - rangeCount, err = ds.tableStats.HistColl.GetRowCountByColumnRanges(ds.ctx, colID, convertedRanges) + rangeCount, err = ds.tableStats.HistColl.GetRowCountByColumnRanges(ds.SCtx(), colID, convertedRanges) } if err != nil { return 0, false, corr @@ -2038,8 +2038,8 @@ func (ds *DataSource) crossEstimateRowCount(path *util.AccessPath, conds []expre // `1 + row_count(a < 1 or a is null)` func (ds *DataSource) crossEstimateIndexRowCount(path *util.AccessPath, expectedCnt float64, desc bool) (float64, bool, float64) { filtersLen := len(path.TableFilters) + len(path.IndexFilters) - sessVars := ds.ctx.GetSessionVars() - if ds.statisticTable.Pseudo || filtersLen == 0 || !sessVars.EnableExtendedStats || !ds.ctx.GetSessionVars().EnableCorrelationAdjustment { + sessVars := ds.SCtx().GetSessionVars() + if ds.statisticTable.Pseudo || filtersLen == 0 || !sessVars.EnableExtendedStats || !ds.SCtx().GetSessionVars().EnableCorrelationAdjustment { return 0, false, 0 } col, corr := getMostCorrCol4Index(path, ds.statisticTable, sessVars.CorrelationThreshold) @@ -2102,8 +2102,8 @@ func (s *LogicalTableScan) GetPhysicalScan(schema *expression.Schema, stats *pro AccessCondition: s.AccessConds, tblCols: ds.TblCols, tblColHists: ds.TblColHists, - }.Init(s.ctx, s.blockOffset) - ts.stats = stats + }.Init(s.SCtx(), s.SelectBlockOffset()) + ts.SetStats(stats) ts.SetSchema(schema.Clone()) if ts.Table.PKIsHandle { if pkColInfo := ts.Table.GetPkColInfo(); pkColInfo != nil { @@ -2133,8 +2133,8 @@ func (s *LogicalIndexScan) GetPhysicalIndexScan(_ *expression.Schema, stats *pro physicalTableID: ds.physicalTableID, tblColHists: ds.TblColHists, pkIsHandleCol: ds.getPKIsHandleCol(), - }.Init(ds.ctx, ds.blockOffset) - is.stats = stats + }.Init(ds.SCtx(), ds.SelectBlockOffset()) + is.SetStats(stats) is.initSchema(s.FullIdxCols, s.IsDoubleRead) return is } @@ -2162,7 +2162,7 @@ func (ds *DataSource) isPointGetPath(path *util.AccessPath) bool { } } for _, ran := range path.Ranges { - if !ran.IsPointNonNullable(ds.ctx) { + if !ran.IsPointNonNullable(ds.SCtx()) { return false } } @@ -2239,12 +2239,12 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid tblColHists: ds.TblColHists, } ts.PartitionInfo = PartitionInfo{ - PruningConds: pushDownNot(ds.ctx, ds.allConds), + PruningConds: pushDownNot(ds.SCtx(), ds.allConds), PartitionNames: ds.partitionNames, Columns: ds.TblCols, ColumnNames: ds.names, } - mppTask = ts.addPushedDownSelectionToMppTask(mppTask, ds.stats.ScaleByExpectCnt(prop.ExpectedCnt)) + mppTask = ts.addPushedDownSelectionToMppTask(mppTask, ds.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt)) task = mppTask if !mppTask.invalid() { if prop.TaskTp == property.MppTaskType && len(mppTask.rootTaskConds) > 0 { @@ -2258,7 +2258,7 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid // which cannot pushdown to tiflash(because TiFlash doesn't support some expr in Proj) // So HashJoin cannot pushdown to tiflash. But we still want TableScan to run on tiflash. task = mppTask - task = task.convertToRootTask(ds.ctx) + task = task.convertToRootTask(ds.SCtx()) } } return task, nil @@ -2273,7 +2273,7 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid tblColHists: ds.TblColHists, } copTask.partitionInfo = PartitionInfo{ - PruningConds: pushDownNot(ds.ctx, ds.allConds), + PruningConds: pushDownNot(ds.SCtx(), ds.allConds), PartitionNames: ds.partitionNames, Columns: ds.TblCols, ColumnNames: ds.names, @@ -2298,12 +2298,12 @@ func (ds *DataSource) convertToTableScan(prop *property.PhysicalProperty, candid ts.ByItems = byItems } } - ts.addPushedDownSelection(copTask, ds.stats.ScaleByExpectCnt(prop.ExpectedCnt)) + ts.addPushedDownSelection(copTask, ds.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt)) if prop.IsFlashProp() && len(copTask.rootTaskConds) != 0 { return invalidTask, nil } if prop.TaskTp == property.RootTaskType { - task = task.convertToRootTask(ds.ctx) + task = task.convertToRootTask(ds.SCtx()) } else if _, ok := task.(*rootTask); ok { return invalidTask, nil } @@ -2328,7 +2328,7 @@ func (ds *DataSource) convertToSampleTable(prop *property.PhysicalProperty, TableSampleInfo: ds.SampleInfo, TableInfo: ds.table, Desc: candidate.isMatchProp && prop.SortItems[0].Desc, - }.Init(ds.ctx, ds.SelectBlockOffset()) + }.Init(ds.SCtx(), ds.SelectBlockOffset()) p.schema = ds.schema return &rootTask{ p: p, @@ -2350,15 +2350,15 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida accessCnt := math.Min(candidate.path.CountAfterAccess, float64(1)) pointGetPlan := PointGetPlan{ - ctx: ds.ctx, + ctx: ds.SCtx(), AccessConditions: candidate.path.AccessConds, schema: ds.schema.Clone(), dbName: ds.DBName.L, TblInfo: ds.TableInfo(), outputNames: ds.OutputNames(), - LockWaitTime: ds.ctx.GetSessionVars().LockWaitTimeout, + LockWaitTime: ds.SCtx().GetSessionVars().LockWaitTimeout, Columns: ds.Columns, - }.Init(ds.ctx, ds.tableStats.ScaleByExpectCnt(accessCnt), ds.blockOffset) + }.Init(ds.SCtx(), ds.tableStats.ScaleByExpectCnt(accessCnt), ds.SelectBlockOffset()) var partitionInfo *model.PartitionDefinition pi := ds.tableInfo.GetPartitionInfo() if ds.isPartition { @@ -2377,7 +2377,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida } } else if pi != nil { // dynamic prune - idxs, err := PartitionPruning(ds.ctx, ds.table.GetPartitionedTable(), ds.allConds, ds.partitionNames, ds.TblCols, ds.names) + idxs, err := PartitionPruning(ds.SCtx(), ds.table.GetPartitionedTable(), ds.allConds, ds.partitionNames, ds.TblCols, ds.names) if err != nil { return invalidTask } @@ -2402,7 +2402,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida if len(candidate.path.TableFilters) > 0 { sel := PhysicalSelection{ Conditions: candidate.path.TableFilters, - }.Init(ds.ctx, ds.stats.ScaleByExpectCnt(prop.ExpectedCnt), ds.blockOffset) + }.Init(ds.SCtx(), ds.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), ds.SelectBlockOffset()) sel.SetChildren(pointGetPlan) rTsk.p = sel } @@ -2421,7 +2421,7 @@ func (ds *DataSource) convertToPointGet(prop *property.PhysicalProperty, candida if len(candidate.path.IndexFilters)+len(candidate.path.TableFilters) > 0 { sel := PhysicalSelection{ Conditions: append(candidate.path.IndexFilters, candidate.path.TableFilters...), - }.Init(ds.ctx, ds.stats.ScaleByExpectCnt(prop.ExpectedCnt), ds.blockOffset) + }.Init(ds.SCtx(), ds.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), ds.SelectBlockOffset()) sel.SetChildren(pointGetPlan) rTsk.p = sel } @@ -2441,7 +2441,7 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca accessCnt := math.Min(candidate.path.CountAfterAccess, float64(len(candidate.path.Ranges))) batchPointGetPlan := &BatchPointGetPlan{ - ctx: ds.ctx, + ctx: ds.SCtx(), dbName: ds.DBName.L, AccessConditions: candidate.path.AccessConds, TblInfo: ds.TableInfo(), @@ -2449,7 +2449,7 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca Columns: ds.Columns, SinglePart: ds.isPartition, PartTblID: ds.physicalTableID, - PartitionExpr: getPartitionExpr(ds.ctx, ds.TableInfo()), + PartitionExpr: getPartitionExpr(ds.SCtx(), ds.TableInfo()), } if batchPointGetPlan.KeepOrder { batchPointGetPlan.Desc = prop.SortItems[0].Desc @@ -2462,10 +2462,10 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca batchPointGetPlan.accessCols = ds.TblCols // Add filter condition to table plan now. if len(candidate.path.TableFilters) > 0 { - batchPointGetPlan.Init(ds.ctx, ds.tableStats.ScaleByExpectCnt(accessCnt), ds.schema.Clone(), ds.names, ds.blockOffset) + batchPointGetPlan.Init(ds.SCtx(), ds.tableStats.ScaleByExpectCnt(accessCnt), ds.schema.Clone(), ds.names, ds.SelectBlockOffset()) sel := PhysicalSelection{ Conditions: candidate.path.TableFilters, - }.Init(ds.ctx, ds.stats.ScaleByExpectCnt(prop.ExpectedCnt), ds.blockOffset) + }.Init(ds.SCtx(), ds.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), ds.SelectBlockOffset()) sel.SetChildren(batchPointGetPlan) rTsk.p = sel } @@ -2488,16 +2488,16 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca } // Add index condition to table plan now. if len(candidate.path.IndexFilters)+len(candidate.path.TableFilters) > 0 { - batchPointGetPlan.Init(ds.ctx, ds.tableStats.ScaleByExpectCnt(accessCnt), ds.schema.Clone(), ds.names, ds.blockOffset) + batchPointGetPlan.Init(ds.SCtx(), ds.tableStats.ScaleByExpectCnt(accessCnt), ds.schema.Clone(), ds.names, ds.SelectBlockOffset()) sel := PhysicalSelection{ Conditions: append(candidate.path.IndexFilters, candidate.path.TableFilters...), - }.Init(ds.ctx, ds.stats.ScaleByExpectCnt(prop.ExpectedCnt), ds.blockOffset) + }.Init(ds.SCtx(), ds.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), ds.SelectBlockOffset()) sel.SetChildren(batchPointGetPlan) rTsk.p = sel } } if rTsk.p == nil { - rTsk.p = batchPointGetPlan.Init(ds.ctx, ds.tableStats.ScaleByExpectCnt(accessCnt), ds.schema.Clone(), ds.names, ds.blockOffset) + rTsk.p = batchPointGetPlan.Init(ds.SCtx(), ds.tableStats.ScaleByExpectCnt(accessCnt), ds.schema.Clone(), ds.names, ds.SelectBlockOffset()) } return rTsk @@ -2506,13 +2506,13 @@ func (ds *DataSource) convertToBatchPointGet(prop *property.PhysicalProperty, ca func (ts *PhysicalTableScan) addPushedDownSelectionToMppTask(mpp *mppTask, stats *property.StatsInfo) *mppTask { filterCondition, rootTaskConds := SplitSelCondsWithVirtualColumn(ts.filterCondition) var newRootConds []expression.Expression - filterCondition, newRootConds = expression.PushDownExprs(ts.ctx.GetSessionVars().StmtCtx, filterCondition, ts.ctx.GetClient(), ts.StoreType) + filterCondition, newRootConds = expression.PushDownExprs(ts.SCtx().GetSessionVars().StmtCtx, filterCondition, ts.SCtx().GetClient(), ts.StoreType) mpp.rootTaskConds = append(rootTaskConds, newRootConds...) ts.filterCondition = filterCondition // Add filter condition to table plan now. if len(ts.filterCondition) > 0 { - sel := PhysicalSelection{Conditions: ts.filterCondition}.Init(ts.ctx, stats, ts.blockOffset) + sel := PhysicalSelection{Conditions: ts.filterCondition}.Init(ts.SCtx(), stats, ts.SelectBlockOffset()) sel.SetChildren(ts) mpp.p = sel } @@ -2522,19 +2522,19 @@ func (ts *PhysicalTableScan) addPushedDownSelectionToMppTask(mpp *mppTask, stats func (ts *PhysicalTableScan) addPushedDownSelection(copTask *copTask, stats *property.StatsInfo) { ts.filterCondition, copTask.rootTaskConds = SplitSelCondsWithVirtualColumn(ts.filterCondition) var newRootConds []expression.Expression - ts.filterCondition, newRootConds = expression.PushDownExprs(ts.ctx.GetSessionVars().StmtCtx, ts.filterCondition, ts.ctx.GetClient(), ts.StoreType) + ts.filterCondition, newRootConds = expression.PushDownExprs(ts.SCtx().GetSessionVars().StmtCtx, ts.filterCondition, ts.SCtx().GetClient(), ts.StoreType) copTask.rootTaskConds = append(copTask.rootTaskConds, newRootConds...) // Add filter condition to table plan now. if len(ts.filterCondition) > 0 { - sel := PhysicalSelection{Conditions: ts.filterCondition}.Init(ts.ctx, stats, ts.blockOffset) + sel := PhysicalSelection{Conditions: ts.filterCondition}.Init(ts.SCtx(), stats, ts.SelectBlockOffset()) if len(copTask.rootTaskConds) != 0 { - selectivity, _, err := copTask.tblColHists.Selectivity(ts.ctx, ts.filterCondition, nil) + selectivity, _, err := copTask.tblColHists.Selectivity(ts.SCtx(), ts.filterCondition, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor } - sel.stats = ts.stats.Scale(selectivity) + sel.SetStats(ts.StatsInfo().Scale(selectivity)) } sel.SetChildren(ts) copTask.tablePlan = sel @@ -2543,11 +2543,11 @@ func (ts *PhysicalTableScan) addPushedDownSelection(copTask *copTask, stats *pro func (ts *PhysicalTableScan) getScanRowSize() float64 { if ts.StoreType == kv.TiKV { - return ts.tblColHists.GetTableAvgRowSize(ts.ctx, ts.tblCols, ts.StoreType, true) + return ts.tblColHists.GetTableAvgRowSize(ts.SCtx(), ts.tblCols, ts.StoreType, true) } // If `ts.handleCol` is nil, then the schema of tableScan doesn't have handle column. // This logic can be ensured in column pruning. - return ts.tblColHists.GetTableAvgRowSize(ts.ctx, ts.Schema().Columns, ts.StoreType, ts.HandleCols != nil) + return ts.tblColHists.GetTableAvgRowSize(ts.SCtx(), ts.Schema().Columns, ts.StoreType, ts.HandleCols != nil) } func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProperty, path *util.AccessPath, isMatchProp bool) (*PhysicalTableScan, float64) { @@ -2566,7 +2566,7 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper tblColHists: ds.TblColHists, constColsByCond: path.ConstCols, prop: prop, - }.Init(ds.ctx, ds.blockOffset) + }.Init(ds.SCtx(), ds.SelectBlockOffset()) ts.filterCondition = make([]expression.Expression, len(path.TableFilters)) copy(ts.filterCondition, path.TableFilters) ts.SetSchema(ds.schema.Clone()) @@ -2578,8 +2578,8 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper } } rowCount := path.CountAfterAccess - if prop.ExpectedCnt < ds.stats.RowCount { - selectivity := ds.stats.RowCount / path.CountAfterAccess + if prop.ExpectedCnt < ds.StatsInfo().RowCount { + selectivity := ds.StatsInfo().RowCount / path.CountAfterAccess uniformEst := math.Min(path.CountAfterAccess, prop.ExpectedCnt/selectivity) corrEst, ok, corr := ds.crossEstimateTableRowCount(path, prop.ExpectedCnt, isMatchProp && prop.SortItems[0].Desc) @@ -2595,7 +2595,7 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper // to reduce risks of correlation adjustment, use the maximum between uniformEst and corrEst rowCount = math.Max(uniformEst, corrEst) } else if abs := math.Abs(corr); abs < 1 { - correlationFactor := math.Pow(1-abs, float64(ds.ctx.GetSessionVars().CorrelationExpFactor)) + correlationFactor := math.Pow(1-abs, float64(ds.SCtx().GetSessionVars().CorrelationExpFactor)) rowCount = math.Min(path.CountAfterAccess, uniformEst/correlationFactor) } } @@ -2604,8 +2604,8 @@ func (ds *DataSource) getOriginalPhysicalTableScan(prop *property.PhysicalProper // to scan, but this would only help improve accuracy of NDV for one column, for other columns, // we still need to assume values are uniformly distributed. For simplicity, we use uniform-assumption // for all columns now, as we do in `deriveStatsByFilter`. - ts.stats = ds.tableStats.ScaleByExpectCnt(rowCount) - usedStats := ds.ctx.GetSessionVars().StmtCtx.GetUsedStatsInfo(false) + ts.SetStats(ds.tableStats.ScaleByExpectCnt(rowCount)) + usedStats := ds.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(false) if usedStats != nil && usedStats[ts.physicalTableID] != nil { ts.usedStatsInfo = usedStats[ts.physicalTableID] } @@ -2635,7 +2635,7 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper pkIsHandleCol: ds.getPKIsHandleCol(), constColsByCond: path.ConstCols, prop: prop, - }.Init(ds.ctx, ds.blockOffset) + }.Init(ds.SCtx(), ds.SelectBlockOffset()) statsTbl := ds.statisticTable if statsTbl.Indices[idx.ID] != nil { is.Hist = &statsTbl.Indices[idx.ID].Histogram @@ -2646,21 +2646,21 @@ func (ds *DataSource) getOriginalPhysicalIndexScan(prop *property.PhysicalProper // If (1) there exists an index whose selectivity is smaller than the threshold, // and (2) there is Selection on the IndexScan, we don't use the ExpectedCnt to // adjust the estimated row count of the IndexScan. - ignoreExpectedCnt := ds.accessPathMinSelectivity < ds.ctx.GetSessionVars().OptOrderingIdxSelThresh && + ignoreExpectedCnt := ds.accessPathMinSelectivity < ds.SCtx().GetSessionVars().OptOrderingIdxSelThresh && len(path.IndexFilters)+len(path.TableFilters) > 0 - if (isMatchProp || prop.IsSortItemEmpty()) && prop.ExpectedCnt < ds.stats.RowCount && !ignoreExpectedCnt { + if (isMatchProp || prop.IsSortItemEmpty()) && prop.ExpectedCnt < ds.StatsInfo().RowCount && !ignoreExpectedCnt { count, ok, corr := ds.crossEstimateIndexRowCount(path, prop.ExpectedCnt, isMatchProp && prop.SortItems[0].Desc) if ok { rowCount = count } else if abs := math.Abs(corr); abs < 1 { - correlationFactor := math.Pow(1-abs, float64(ds.ctx.GetSessionVars().CorrelationExpFactor)) - selectivity := ds.stats.RowCount / rowCount + correlationFactor := math.Pow(1-abs, float64(ds.SCtx().GetSessionVars().CorrelationExpFactor)) + selectivity := ds.StatsInfo().RowCount / rowCount rowCount = math.Min(prop.ExpectedCnt/selectivity/correlationFactor, rowCount) } } - is.stats = ds.tableStats.ScaleByExpectCnt(rowCount) - usedStats := ds.ctx.GetSessionVars().StmtCtx.GetUsedStatsInfo(false) + is.SetStats(ds.tableStats.ScaleByExpectCnt(rowCount)) + usedStats := ds.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(false) if usedStats != nil && usedStats[is.physicalTableID] != nil { is.usedStatsInfo = usedStats[is.physicalTableID] } @@ -2679,10 +2679,10 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *Plan return invalidTask, 1, nil } // The physical plan has been build when derive stats. - pcte := PhysicalCTE{SeedPlan: p.cte.seedPartPhysicalPlan, RecurPlan: p.cte.recursivePartPhysicalPlan, CTE: p.cte, cteAsName: p.cteAsName, cteName: p.cteName}.Init(p.ctx, p.stats) + pcte := PhysicalCTE{SeedPlan: p.cte.seedPartPhysicalPlan, RecurPlan: p.cte.recursivePartPhysicalPlan, CTE: p.cte, cteAsName: p.cteAsName, cteName: p.cteName}.Init(p.SCtx(), p.StatsInfo()) pcte.SetSchema(p.schema) if prop.IsFlashProp() && prop.CTEProducerStatus == property.AllCTECanMpp { - pcte.readerReceiver = PhysicalExchangeReceiver{IsCTEReader: true}.Init(p.ctx, p.stats) + pcte.readerReceiver = PhysicalExchangeReceiver{IsCTEReader: true}.Init(p.SCtx(), p.StatsInfo()) if prop.MPPPartitionTp != property.AnyType { return invalidTask, 1, nil } @@ -2690,13 +2690,13 @@ func (p *LogicalCTE) findBestTask(prop *property.PhysicalProperty, counter *Plan p: pcte, partTp: prop.MPPPartitionTp, hashCols: prop.MPPPartitionCols, - tblColHists: p.stats.HistColl, + tblColHists: p.StatsInfo().HistColl, } } else { t = &rootTask{p: pcte, isEmpty: false} } if prop.CanAddEnforcer { - t = enforceProperty(prop, t, p.basePlan.ctx) + t = enforceProperty(prop, t, p.Plan.SCtx()) } return t, 1, nil } @@ -2706,7 +2706,7 @@ func (p *LogicalCTETable) findBestTask(prop *property.PhysicalProperty, _ *PlanC return nil, 1, nil } - pcteTable := PhysicalCTETable{IDForStorage: p.idForStorage}.Init(p.ctx, p.stats) + pcteTable := PhysicalCTETable{IDForStorage: p.idForStorage}.Init(p.SCtx(), p.StatsInfo()) pcteTable.SetSchema(p.schema) t = &rootTask{p: pcteTable} return t, 1, nil diff --git a/planner/core/find_best_task_test.go b/planner/core/find_best_task_test.go index e9067853a8f21..7eac1e0f7e6d5 100644 --- a/planner/core/find_best_task_test.go +++ b/planner/core/find_best_task_test.go @@ -36,7 +36,7 @@ func (ds mockDataSource) Init(ctx sessionctx.Context) *mockDataSource { func (ds *mockDataSource) findBestTask(prop *property.PhysicalProperty, planCounter *PlanCounterTp, opt *physicalOptimizeOp) (task, int64, error) { // It can satisfy any of the property! // Just use a TableDual for convenience. - p := PhysicalTableDual{}.Init(ds.ctx, &property.StatsInfo{RowCount: 1}, 0) + p := PhysicalTableDual{}.Init(ds.SCtx(), &property.StatsInfo{RowCount: 1}, 0) task := &rootTask{ p: p, } @@ -71,16 +71,16 @@ func (p mockLogicalPlan4Test) Init(ctx sessionctx.Context) *mockLogicalPlan4Test } func (p *mockLogicalPlan4Test) getPhysicalPlan1(prop *property.PhysicalProperty) PhysicalPlan { - physicalPlan1 := mockPhysicalPlan4Test{planType: 1}.Init(p.ctx) - physicalPlan1.stats = &property.StatsInfo{RowCount: 1} + physicalPlan1 := mockPhysicalPlan4Test{planType: 1}.Init(p.SCtx()) + physicalPlan1.SetStats(&property.StatsInfo{RowCount: 1}) physicalPlan1.childrenReqProps = make([]*property.PhysicalProperty, 1) physicalPlan1.childrenReqProps[0] = prop.CloneEssentialFields() return physicalPlan1 } func (p *mockLogicalPlan4Test) getPhysicalPlan2(prop *property.PhysicalProperty) PhysicalPlan { - physicalPlan2 := mockPhysicalPlan4Test{planType: 2}.Init(p.ctx) - physicalPlan2.stats = &property.StatsInfo{RowCount: 1} + physicalPlan2 := mockPhysicalPlan4Test{planType: 2}.Init(p.SCtx()) + physicalPlan2.SetStats(&property.StatsInfo{RowCount: 1}) physicalPlan2.childrenReqProps = make([]*property.PhysicalProperty, 1) physicalPlan2.childrenReqProps[0] = property.NewPhysicalProperty(prop.TaskTp, nil, false, prop.ExpectedCnt, false) return physicalPlan2 @@ -103,7 +103,7 @@ func (p *mockLogicalPlan4Test) exhaustPhysicalPlans(prop *property.PhysicalPrope if p.hasHintForPlan2 { // The hint cannot work. if prop.IsSortItemEmpty() { - p.ctx.GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("the hint is inapplicable for plan2")) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(fmt.Errorf("the hint is inapplicable for plan2")) } return plan1, false, nil } diff --git a/planner/core/fragment.go b/planner/core/fragment.go index 79cbead1c6a8c..4171eb592ae22 100644 --- a/planner/core/fragment.go +++ b/planner/core/fragment.go @@ -460,7 +460,7 @@ func (e *mppTaskGenerator) generateTasksForCTEReader(cteReader *PhysicalCTE) (er for i, col := range cols { col.Index = i } - proj := PhysicalProjection{Exprs: expression.Column2Exprs(cols)}.Init(cteReader.ctx, cteReader.stats, 0, nil) + proj := PhysicalProjection{Exprs: expression.Column2Exprs(cols)}.Init(cteReader.SCtx(), cteReader.StatsInfo(), 0, nil) proj.SetSchema(cteReader.schema.Clone()) proj.SetChildren(receiver) cteReader.SetChildren(proj) diff --git a/planner/core/hashcode.go b/planner/core/hashcode.go index 41b00de127390..d9f49086eb8b0 100644 --- a/planner/core/hashcode.go +++ b/planner/core/hashcode.go @@ -33,7 +33,7 @@ func (p *baseLogicalPlan) HashCode() []byte { // We use PlanID for the default hash, so if two plans do not have // the same id, the hash value will never be the same. result := make([]byte, 0, 4) - result = encodeIntAsUint32(result, p.id) + result = encodeIntAsUint32(result, p.ID()) return result } @@ -43,11 +43,11 @@ func (p *LogicalProjection) HashCode() []byte { // Expressions are commonly `Column`s, whose hashcode has the length 9, so // we pre-alloc 10 bytes for each expr's hashcode. result := make([]byte, 0, 12+len(p.Exprs)*10) - result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.tp)) + result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP())) result = encodeIntAsUint32(result, p.SelectBlockOffset()) result = encodeIntAsUint32(result, len(p.Exprs)) for _, expr := range p.Exprs { - exprHashCode := expr.HashCode(p.ctx.GetSessionVars().StmtCtx) + exprHashCode := expr.HashCode(p.SCtx().GetSessionVars().StmtCtx) result = encodeIntAsUint32(result, len(exprHashCode)) result = append(result, exprHashCode...) } @@ -58,7 +58,7 @@ func (p *LogicalProjection) HashCode() []byte { func (p *LogicalTableDual) HashCode() []byte { // PlanType + SelectOffset + RowCount result := make([]byte, 0, 12) - result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.tp)) + result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP())) result = encodeIntAsUint32(result, p.SelectBlockOffset()) result = encodeIntAsUint32(result, p.RowCount) return result @@ -70,13 +70,13 @@ func (p *LogicalSelection) HashCode() []byte { // Conditions are commonly `ScalarFunction`s, whose hashcode usually has a // length larger than 20, so we pre-alloc 25 bytes for each expr's hashcode. result := make([]byte, 0, 12+len(p.Conditions)*25) - result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.tp)) + result = encodeIntAsUint32(result, plancodec.TypeStringToPhysicalID(p.TP())) result = encodeIntAsUint32(result, p.SelectBlockOffset()) result = encodeIntAsUint32(result, len(p.Conditions)) condHashCodes := make([][]byte, len(p.Conditions)) for i, expr := range p.Conditions { - condHashCodes[i] = expr.HashCode(p.ctx.GetSessionVars().StmtCtx) + condHashCodes[i] = expr.HashCode(p.SCtx().GetSessionVars().StmtCtx) } // Sort the conditions, so `a > 1 and a < 100` can equal to `a < 100 and a > 1`. slices.SortFunc(condHashCodes, func(i, j []byte) bool { return bytes.Compare(i, j) < 0 }) @@ -92,7 +92,7 @@ func (p *LogicalSelection) HashCode() []byte { func (p *LogicalLimit) HashCode() []byte { // PlanType + SelectOffset + Offset + Count result := make([]byte, 24) - binary.BigEndian.PutUint32(result, uint32(plancodec.TypeStringToPhysicalID(p.tp))) + binary.BigEndian.PutUint32(result, uint32(plancodec.TypeStringToPhysicalID(p.TP()))) binary.BigEndian.PutUint32(result[4:], uint32(p.SelectBlockOffset())) binary.BigEndian.PutUint64(result[8:], p.Offset) binary.BigEndian.PutUint64(result[16:], p.Count) diff --git a/planner/core/indexmerge_path.go b/planner/core/indexmerge_path.go index 102d1b4995bf7..e2b56f2bd1b86 100644 --- a/planner/core/indexmerge_path.go +++ b/planner/core/indexmerge_path.go @@ -37,12 +37,12 @@ import ( // generateIndexMergePath generates IndexMerge AccessPaths on this DataSource. func (ds *DataSource) generateIndexMergePath() error { - if ds.ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugtrace.EnterContextCommon(ds.ctx) - defer debugtrace.LeaveContextCommon(ds.ctx) + if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(ds.SCtx()) + defer debugtrace.LeaveContextCommon(ds.SCtx()) } var warningMsg string - stmtCtx := ds.ctx.GetSessionVars().StmtCtx + stmtCtx := ds.SCtx().GetSessionVars().StmtCtx defer func() { if len(ds.indexMergeHints) > 0 && warningMsg != "" { ds.indexMergeHints = nil @@ -57,10 +57,10 @@ func (ds *DataSource) generateIndexMergePath() error { // We will create new Selection for exprs that cannot be pushed in convertToIndexMergeScan. indexMergeConds := make([]expression.Expression, 0, len(ds.allConds)) for _, expr := range ds.allConds { - indexMergeConds = append(indexMergeConds, expression.PushDownNot(ds.ctx, expr)) + indexMergeConds = append(indexMergeConds, expression.PushDownNot(ds.SCtx(), expr)) } - sessionAndStmtPermission := (ds.ctx.GetSessionVars().GetEnableIndexMerge() || len(ds.indexMergeHints) > 0) && !stmtCtx.NoIndexMergeHint + sessionAndStmtPermission := (ds.SCtx().GetSessionVars().GetEnableIndexMerge() || len(ds.indexMergeHints) > 0) && !stmtCtx.NoIndexMergeHint if !sessionAndStmtPermission { warningMsg = "IndexMerge is inapplicable or disabled. Got no_index_merge hint or tidb_enable_index_merge is off." return nil @@ -100,8 +100,8 @@ func (ds *DataSource) generateIndexMergePath() error { minRowCount = path.CountAfterAccess } } - if ds.stats.RowCount > minRowCount { - ds.stats = ds.tableStats.ScaleByExpectCnt(minRowCount) + if ds.StatsInfo().RowCount > minRowCount { + ds.SetStats(ds.tableStats.ScaleByExpectCnt(minRowCount)) } return nil } @@ -124,9 +124,9 @@ func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) pushedDownCNFItems := make([]expression.Expression, 0, len(cnfItems)) for _, cnfItem := range cnfItems { - if expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, + if expression.CanExprsPushDown(ds.SCtx().GetSessionVars().StmtCtx, []expression.Expression{cnfItem}, - ds.ctx.GetClient(), + ds.SCtx().GetClient(), kv.TiKV, ) { pushedDownCNFItems = append(pushedDownCNFItems, cnfItem) @@ -172,11 +172,11 @@ func (ds *DataSource) generateIndexMergeOrPaths(filters []expression.Expression) indexCondsForP := p.AccessConds[:] indexCondsForP = append(indexCondsForP, p.IndexFilters...) if len(indexCondsForP) > 0 { - accessConds = append(accessConds, expression.ComposeCNFCondition(ds.ctx, indexCondsForP...)) + accessConds = append(accessConds, expression.ComposeCNFCondition(ds.SCtx(), indexCondsForP...)) } } - accessDNF := expression.ComposeDNFCondition(ds.ctx, accessConds...) - sel, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, []expression.Expression{accessDNF}, nil) + accessDNF := expression.ComposeDNFCondition(ds.SCtx(), accessConds...) + sel, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), []expression.Expression{accessDNF}, nil) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) sel = SelectionFactor @@ -343,7 +343,7 @@ func (ds *DataSource) buildIndexMergeOrPath( // Global index is not compatible with IndexMergeReaderExecutor. for i := range partialPaths { if partialPaths[i].Index != nil && partialPaths[i].Index.Global { - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(errors.New("global index is not compatible with index merge, so ignore it")) + ds.SCtx().GetSessionVars().StmtCtx.AppendWarning(errors.New("global index is not compatible with index merge, so ignore it")) return nil } } @@ -354,19 +354,19 @@ func (ds *DataSource) buildIndexMergeOrPath( shouldKeepCurrentFilter = true } // If any partial path's index filter cannot be pushed to TiKV, we should keep the whole DNF filter. - if len(path.IndexFilters) != 0 && !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, path.IndexFilters, ds.ctx.GetClient(), kv.TiKV) { + if len(path.IndexFilters) != 0 && !expression.CanExprsPushDown(ds.SCtx().GetSessionVars().StmtCtx, path.IndexFilters, ds.SCtx().GetClient(), kv.TiKV) { shouldKeepCurrentFilter = true // Clear IndexFilter, the whole filter will be put in indexMergePath.TableFilters. path.IndexFilters = nil } - if len(path.TableFilters) != 0 && !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, path.TableFilters, ds.ctx.GetClient(), kv.TiKV) { + if len(path.TableFilters) != 0 && !expression.CanExprsPushDown(ds.SCtx().GetSessionVars().StmtCtx, path.TableFilters, ds.SCtx().GetClient(), kv.TiKV) { shouldKeepCurrentFilter = true path.TableFilters = nil } } // Keep this filter as a part of table filters for safety if it has any parameter. - if expression.MaybeOverOptimized4PlanCache(ds.ctx, filters[current:current+1]) { + if expression.MaybeOverOptimized4PlanCache(ds.SCtx(), filters[current:current+1]) { shouldKeepCurrentFilter = true } if shouldKeepCurrentFilter { @@ -416,7 +416,7 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int) *util.Access coveredConds = append(coveredConds, path.AccessConds...) for i, cond := range path.IndexFilters { // IndexFilters can be covered by partial path if it can be pushed down to TiKV. - if !expression.CanExprsPushDown(ds.ctx.GetSessionVars().StmtCtx, []expression.Expression{cond}, ds.ctx.GetClient(), kv.TiKV) { + if !expression.CanExprsPushDown(ds.SCtx().GetSessionVars().StmtCtx, []expression.Expression{cond}, ds.SCtx().GetClient(), kv.TiKV) { path.IndexFilters = append(path.IndexFilters[:i], path.IndexFilters[i+1:]...) notCoveredConds = append(notCoveredConds, cond) } else { @@ -432,11 +432,11 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int) *util.Access // avoid wrong deduplication. notCoveredHashCodeSet := make(map[string]struct{}) for _, cond := range notCoveredConds { - hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) + hashCode := string(cond.HashCode(ds.SCtx().GetSessionVars().StmtCtx)) notCoveredHashCodeSet[hashCode] = struct{}{} } for _, cond := range coveredConds { - hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) + hashCode := string(cond.HashCode(ds.SCtx().GetSessionVars().StmtCtx)) if _, ok := notCoveredHashCodeSet[hashCode]; !ok { hashCodeSet[hashCode] = struct{}{} } @@ -449,7 +449,7 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int) *util.Access // Remove covered filters from finalFilters and deduplicate finalFilters. dedupedFinalFilters := make([]expression.Expression, 0, len(finalFilters)) for _, cond := range finalFilters { - hashCode := string(cond.HashCode(ds.ctx.GetSessionVars().StmtCtx)) + hashCode := string(cond.HashCode(ds.SCtx().GetSessionVars().StmtCtx)) if _, ok := hashCodeSet[hashCode]; !ok { dedupedFinalFilters = append(dedupedFinalFilters, cond) hashCodeSet[hashCode] = struct{}{} @@ -457,12 +457,12 @@ func (ds *DataSource) generateIndexMergeAndPaths(normalPathCnt int) *util.Access } // Keep these partial filters as a part of table filters for safety if there is any parameter. - if expression.MaybeOverOptimized4PlanCache(ds.ctx, partialFilters) { + if expression.MaybeOverOptimized4PlanCache(ds.SCtx(), partialFilters) { dedupedFinalFilters = append(dedupedFinalFilters, partialFilters...) } // 3. Estimate the row count after partial paths. - sel, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, partialFilters, nil) + sel, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), partialFilters, nil) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) sel = SelectionFactor @@ -487,7 +487,7 @@ func (ds *DataSource) generateIndexMerge4NormalIndex(regularPathCount int, index // We current do not consider `IndexMergePath`: // 1. If there is an index path. // 2. TODO: If there exists exprs that cannot be pushed down. This is to avoid wrongly estRow of Selection added by rule_predicate_push_down. - stmtCtx := ds.ctx.GetSessionVars().StmtCtx + stmtCtx := ds.SCtx().GetSessionVars().StmtCtx needConsiderIndexMerge := true if len(ds.indexMergeHints) == 0 { for i := 1; i < len(ds.possibleAccessPaths); i++ { @@ -500,7 +500,7 @@ func (ds *DataSource) generateIndexMerge4NormalIndex(regularPathCount int, index // PushDownExprs() will append extra warnings, which is annoying. So we reset warnings here. warnings := stmtCtx.GetWarnings() extraWarnings := stmtCtx.GetExtraWarnings() - _, remaining := expression.PushDownExprs(stmtCtx, indexMergeConds, ds.ctx.GetClient(), kv.UnSpecified) + _, remaining := expression.PushDownExprs(stmtCtx, indexMergeConds, ds.SCtx().GetClient(), kv.UnSpecified) stmtCtx.SetWarnings(warnings) stmtCtx.SetExtraWarnings(extraWarnings) if len(remaining) > 0 { @@ -712,21 +712,21 @@ func (ds *DataSource) buildPartialPaths4MVIndex(accessFilters []expression.Expre virColVals = append(virColVals, v) case ast.JSONContains: // (json_contains(a->'$.zip', '[1, 2, 3]') isIntersection = true - virColVals, ok = jsonArrayExpr2Exprs(ds.ctx, sf.GetArgs()[1], jsonType) + virColVals, ok = jsonArrayExpr2Exprs(ds.SCtx(), sf.GetArgs()[1], jsonType) if !ok || len(virColVals) == 0 { // json_contains(JSON, '[]') is TRUE return nil, false, false, nil } case ast.JSONOverlaps: // (json_overlaps(a->'$.zip', '[1, 2, 3]') var jsonPathIdx int - if sf.GetArgs()[0].Equal(ds.ctx, targetJSONPath) { + if sf.GetArgs()[0].Equal(ds.SCtx(), targetJSONPath) { jsonPathIdx = 0 // (json_overlaps(a->'$.zip', '[1, 2, 3]') - } else if sf.GetArgs()[1].Equal(ds.ctx, targetJSONPath) { + } else if sf.GetArgs()[1].Equal(ds.SCtx(), targetJSONPath) { jsonPathIdx = 1 // (json_overlaps('[1, 2, 3]', a->'$.zip') } else { return nil, false, false, nil } var ok bool - virColVals, ok = jsonArrayExpr2Exprs(ds.ctx, sf.GetArgs()[1-jsonPathIdx], jsonType) + virColVals, ok = jsonArrayExpr2Exprs(ds.SCtx(), sf.GetArgs()[1-jsonPathIdx], jsonType) if !ok || len(virColVals) == 0 { // forbid empty array for safety return nil, false, false, nil } @@ -736,7 +736,7 @@ func (ds *DataSource) buildPartialPaths4MVIndex(accessFilters []expression.Expre for _, v := range virColVals { // rewrite json functions to EQ to calculate range, `(1 member of j)` -> `j=1`. - eq, err := expression.NewFunction(ds.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), virCol, v) + eq, err := expression.NewFunction(ds.SCtx(), ast.EQ, types.NewFieldType(mysql.TypeTiny), virCol, v) if err != nil { return nil, false, false, err } @@ -844,12 +844,12 @@ func (ds *DataSource) checkFilter4MVIndexColumn(filter expression.Expression, id } switch sf.FuncName.L { case ast.JSONMemberOf: // (1 member of a) - return targetJSONPath.Equal(ds.ctx, sf.GetArgs()[1]) + return targetJSONPath.Equal(ds.SCtx(), sf.GetArgs()[1]) case ast.JSONContains: // json_contains(a, '1') - return targetJSONPath.Equal(ds.ctx, sf.GetArgs()[0]) + return targetJSONPath.Equal(ds.SCtx(), sf.GetArgs()[0]) case ast.JSONOverlaps: // json_overlaps(a, '1') or json_overlaps('1', a) - return targetJSONPath.Equal(ds.ctx, sf.GetArgs()[0]) || - targetJSONPath.Equal(ds.ctx, sf.GetArgs()[1]) + return targetJSONPath.Equal(ds.SCtx(), sf.GetArgs()[0]) || + targetJSONPath.Equal(ds.SCtx(), sf.GetArgs()[1]) default: return false } @@ -872,7 +872,7 @@ func (ds *DataSource) checkFilter4MVIndexColumn(filter expression.Expression, id if argCol == nil || argConst == nil { return false } - if argCol.Equal(ds.ctx, idxCol) { + if argCol.Equal(ds.SCtx(), idxCol) { return true } } diff --git a/planner/core/indexmerge_test.go b/planner/core/indexmerge_test.go index 8867e5a64c744..44c3e123b30cd 100644 --- a/planner/core/indexmerge_test.go +++ b/planner/core/indexmerge_test.go @@ -93,7 +93,7 @@ func TestIndexMergePathGeneration(t *testing.T) { lp = lp.Children()[0] } } - ds.ctx.GetSessionVars().SetEnableIndexMerge(true) + ds.SCtx().GetSessionVars().SetEnableIndexMerge(true) idxMergeStartIndex := len(ds.possibleAccessPaths) _, err = lp.recursiveDeriveStats(nil) require.NoError(t, err) diff --git a/planner/core/initialize.go b/planner/core/initialize.go index 8c111a7a4a959..09cd4386cd098 100644 --- a/planner/core/initialize.go +++ b/planner/core/initialize.go @@ -17,6 +17,7 @@ package core import ( "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/planner/core/internal/base" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/types" @@ -77,7 +78,7 @@ func (p LogicalSelection) Init(ctx sessionctx.Context, offset int) *LogicalSelec func (p PhysicalSelection) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalSelection { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSel, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -103,7 +104,7 @@ func (p LogicalExpand) Init(ctx sessionctx.Context, offset int) *LogicalExpand { func (p PhysicalProjection) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalProjection { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeProj, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -123,7 +124,7 @@ func (p LogicalPartitionUnionAll) Init(ctx sessionctx.Context, offset int) *Logi func (p PhysicalUnionAll) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalUnionAll { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeUnion, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -137,7 +138,7 @@ func (ls LogicalSort) Init(ctx sessionctx.Context, offset int) *LogicalSort { func (p PhysicalSort) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalSort { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSort, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -145,7 +146,7 @@ func (p PhysicalSort) Init(ctx sessionctx.Context, stats *property.StatsInfo, of func (p NominalSort) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *NominalSort { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSort, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -159,7 +160,7 @@ func (lt LogicalTopN) Init(ctx sessionctx.Context, offset int) *LogicalTopN { func (p PhysicalTopN) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalTopN { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeTopN, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -173,7 +174,7 @@ func (p LogicalLimit) Init(ctx sessionctx.Context, offset int) *LogicalLimit { func (p PhysicalLimit) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalLimit { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeLimit, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -186,7 +187,7 @@ func (p LogicalTableDual) Init(ctx sessionctx.Context, offset int) *LogicalTable // Init initializes PhysicalTableDual. func (p PhysicalTableDual) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int) *PhysicalTableDual { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeDual, &p, offset) - p.stats = stats + p.SetStats(stats) return &p } @@ -200,7 +201,7 @@ func (p LogicalMaxOneRow) Init(ctx sessionctx.Context, offset int) *LogicalMaxOn func (p PhysicalMaxOneRow) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalMaxOneRow { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeMaxOneRow, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -214,7 +215,7 @@ func (p LogicalWindow) Init(ctx sessionctx.Context, offset int) *LogicalWindow { func (p PhysicalWindow) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalWindow { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeWindow, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -222,7 +223,7 @@ func (p PhysicalWindow) Init(ctx sessionctx.Context, stats *property.StatsInfo, func (p PhysicalShuffle) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalShuffle { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeShuffle, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -230,37 +231,37 @@ func (p PhysicalShuffle) Init(ctx sessionctx.Context, stats *property.StatsInfo, func (p PhysicalShuffleReceiverStub) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalShuffleReceiverStub { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeShuffleReceiver, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } // Init initializes Update. func (p Update) Init(ctx sessionctx.Context) *Update { - p.basePlan = newBasePlan(ctx, plancodec.TypeUpdate, 0) + p.Plan = base.NewBasePlan(ctx, plancodec.TypeUpdate, 0) return &p } // Init initializes Delete. func (p Delete) Init(ctx sessionctx.Context) *Delete { - p.basePlan = newBasePlan(ctx, plancodec.TypeDelete, 0) + p.Plan = base.NewBasePlan(ctx, plancodec.TypeDelete, 0) return &p } // Init initializes Insert. func (p Insert) Init(ctx sessionctx.Context) *Insert { - p.basePlan = newBasePlan(ctx, plancodec.TypeInsert, 0) + p.Plan = base.NewBasePlan(ctx, plancodec.TypeInsert, 0) return &p } // Init initializes LoadData. func (p LoadData) Init(ctx sessionctx.Context) *LoadData { - p.basePlan = newBasePlan(ctx, plancodec.TypeLoadData, 0) + p.Plan = base.NewBasePlan(ctx, plancodec.TypeLoadData, 0) return &p } // Init initializes ImportInto. func (p ImportInto) Init(ctx sessionctx.Context) *ImportInto { - p.basePlan = newBasePlan(ctx, plancodec.TypeImportInto, 0) + p.Plan = base.NewBasePlan(ctx, plancodec.TypeImportInto, 0) return &p } @@ -280,7 +281,7 @@ func (p LogicalShowDDLJobs) Init(ctx sessionctx.Context) *LogicalShowDDLJobs { func (p PhysicalShow) Init(ctx sessionctx.Context) *PhysicalShow { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeShow, &p, 0) // Just use pseudo stats to avoid panic. - p.stats = &property.StatsInfo{RowCount: 1} + p.SetStats(&property.StatsInfo{RowCount: 1}) return &p } @@ -288,7 +289,7 @@ func (p PhysicalShow) Init(ctx sessionctx.Context) *PhysicalShow { func (p PhysicalShowDDLJobs) Init(ctx sessionctx.Context) *PhysicalShowDDLJobs { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeShowDDLJobs, &p, 0) // Just use pseudo stats to avoid panic. - p.stats = &property.StatsInfo{RowCount: 1} + p.SetStats(&property.StatsInfo{RowCount: 1}) return &p } @@ -302,7 +303,7 @@ func (p LogicalLock) Init(ctx sessionctx.Context) *LogicalLock { func (p PhysicalLock) Init(ctx sessionctx.Context, stats *property.StatsInfo, props ...*property.PhysicalProperty) *PhysicalLock { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeLock, &p, 0) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -327,7 +328,7 @@ func (p LogicalMemTable) Init(ctx sessionctx.Context, offset int) *LogicalMemTab // Init initializes PhysicalMemTable. func (p PhysicalMemTable) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int) *PhysicalMemTable { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeMemTableScan, &p, offset) - p.stats = stats + p.SetStats(stats) return &p } @@ -336,21 +337,21 @@ func (p PhysicalHashJoin) Init(ctx sessionctx.Context, stats *property.StatsInfo tp := plancodec.TypeHashJoin p.basePhysicalPlan = newBasePhysicalPlan(ctx, tp, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } // Init initializes PhysicalMergeJoin. func (p PhysicalMergeJoin) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int) *PhysicalMergeJoin { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeMergeJoin, &p, offset) - p.stats = stats + p.SetStats(stats) return &p } // Init initializes basePhysicalAgg. func (base basePhysicalAgg) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int) *basePhysicalAgg { base.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeHashAgg, &base, offset) - base.stats = stats + base.SetStats(stats) return &base } @@ -358,7 +359,7 @@ func (base basePhysicalAgg) initForHash(ctx sessionctx.Context, stats *property. p := &PhysicalHashAgg{base} p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeHashAgg, p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return p } @@ -366,7 +367,7 @@ func (base basePhysicalAgg) initForStream(ctx sessionctx.Context, stats *propert p := &PhysicalStreamAgg{base} p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeStreamAgg, p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return p } @@ -374,7 +375,7 @@ func (base basePhysicalAgg) initForStream(ctx sessionctx.Context, stats *propert func (p PhysicalApply) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalApply { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeApply, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -382,7 +383,7 @@ func (p PhysicalApply) Init(ctx sessionctx.Context, stats *property.StatsInfo, o func (p PhysicalUnionScan) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalUnionScan { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeUnionScan, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -399,14 +400,14 @@ func (p PhysicalIndexLookUpReader) Init(ctx sessionctx.Context, offset int) *Phy func (p PhysicalIndexMergeReader) Init(ctx sessionctx.Context, offset int) *PhysicalIndexMergeReader { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeIndexMerge, &p, offset) if p.tablePlan != nil { - p.stats = p.tablePlan.statsInfo() + p.SetStats(p.tablePlan.StatsInfo()) } else { var totalRowCount float64 for _, partPlan := range p.partialPlans { totalRowCount += partPlan.StatsCount() } - p.stats = p.partialPlans[0].statsInfo().ScaleByExpectCnt(totalRowCount) - p.stats.StatsVersion = p.partialPlans[0].statsInfo().StatsVersion + p.SetStats(p.partialPlans[0].StatsInfo().ScaleByExpectCnt(totalRowCount)) + p.StatsInfo().StatsVersion = p.partialPlans[0].StatsInfo().StatsVersion } p.PartialPlans = make([][]PhysicalPlan, 0, len(p.partialPlans)) for _, partialPlan := range p.partialPlans { @@ -482,7 +483,7 @@ func (p PhysicalTableReader) Init(ctx sessionctx.Context, offset int) *PhysicalT // Init initializes PhysicalTableSample. func (p PhysicalTableSample) Init(ctx sessionctx.Context, offset int) *PhysicalTableSample { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeTableSample, &p, offset) - p.stats = &property.StatsInfo{RowCount: 1} + p.SetStats(&property.StatsInfo{RowCount: 1}) return &p } @@ -510,34 +511,34 @@ func (p PhysicalIndexReader) Init(ctx sessionctx.Context, offset int) *PhysicalI func (p PhysicalIndexJoin) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalIndexJoin { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeIndexJoin, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } // Init initializes PhysicalIndexMergeJoin. func (p PhysicalIndexMergeJoin) Init(ctx sessionctx.Context) *PhysicalIndexMergeJoin { - p.tp = plancodec.TypeIndexMergeJoin - p.id = int(ctx.GetSessionVars().PlanID.Add(1)) - p.ctx = ctx + p.SetTP(plancodec.TypeIndexMergeJoin) + p.SetID(int(ctx.GetSessionVars().PlanID.Add(1))) + p.SetSCtx(ctx) p.self = &p return &p } // Init initializes PhysicalIndexHashJoin. func (p PhysicalIndexHashJoin) Init(ctx sessionctx.Context) *PhysicalIndexHashJoin { - p.tp = plancodec.TypeIndexHashJoin - p.id = int(ctx.GetSessionVars().PlanID.Add(1)) - p.ctx = ctx + p.SetTP(plancodec.TypeIndexHashJoin) + p.SetID(int(ctx.GetSessionVars().PlanID.Add(1))) + p.SetSCtx(ctx) p.self = &p return &p } // Init initializes BatchPointGetPlan. func (p *BatchPointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsInfo, schema *expression.Schema, names []*types.FieldName, offset int) *BatchPointGetPlan { - p.basePlan = newBasePlan(ctx, plancodec.TypeBatchPointGet, offset) + p.Plan = base.NewBasePlan(ctx, plancodec.TypeBatchPointGet, offset) p.schema = schema p.names = names - p.stats = stats + p.SetStats(stats) p.Columns = ExpandVirtualColumn(p.Columns, p.schema, p.TblInfo.Columns) var ( @@ -585,23 +586,23 @@ func (p *BatchPointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsIn // Init initializes PointGetPlan. func (p PointGetPlan) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, _ ...*property.PhysicalProperty) *PointGetPlan { - p.basePlan = newBasePlan(ctx, plancodec.TypePointGet, offset) - p.stats = stats + p.Plan = base.NewBasePlan(ctx, plancodec.TypePointGet, offset) + p.SetStats(stats) p.Columns = ExpandVirtualColumn(p.Columns, p.schema, p.TblInfo.Columns) return &p } // Init only assigns type and context. func (p PhysicalExchangeSender) Init(ctx sessionctx.Context, stats *property.StatsInfo) *PhysicalExchangeSender { - p.basePlan = newBasePlan(ctx, plancodec.TypeExchangeSender, 0) - p.stats = stats + p.Plan = base.NewBasePlan(ctx, plancodec.TypeExchangeSender, 0) + p.SetStats(stats) return &p } // Init only assigns type and context. func (p PhysicalExchangeReceiver) Init(ctx sessionctx.Context, stats *property.StatsInfo) *PhysicalExchangeReceiver { - p.basePlan = newBasePlan(ctx, plancodec.TypeExchangeReceiver, 0) - p.stats = stats + p.Plan = base.NewBasePlan(ctx, plancodec.TypeExchangeReceiver, 0) + p.SetStats(stats) return &p } @@ -633,7 +634,7 @@ func (p LogicalCTE) Init(ctx sessionctx.Context, offset int) *LogicalCTE { // Init only assigns type and context. func (p PhysicalCTE) Init(ctx sessionctx.Context, stats *property.StatsInfo) *PhysicalCTE { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeCTE, &p, 0) - p.stats = stats + p.SetStats(stats) return &p } @@ -645,22 +646,22 @@ func (p LogicalCTETable) Init(ctx sessionctx.Context, offset int) *LogicalCTETab // Init only assigns type and context. func (p PhysicalCTETable) Init(ctx sessionctx.Context, stats *property.StatsInfo) *PhysicalCTETable { - p.basePlan = newBasePlan(ctx, plancodec.TypeCTETable, 0) - p.stats = stats + p.Plan = base.NewBasePlan(ctx, plancodec.TypeCTETable, 0) + p.SetStats(stats) return &p } // Init initializes FKCheck. func (p FKCheck) Init(ctx sessionctx.Context) *FKCheck { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeForeignKeyCheck, &p, 0) - p.stats = &property.StatsInfo{} + p.SetStats(&property.StatsInfo{}) return &p } // Init initializes FKCascade func (p FKCascade) Init(ctx sessionctx.Context) *FKCascade { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeForeignKeyCascade, &p, 0) - p.stats = &property.StatsInfo{} + p.SetStats(&property.StatsInfo{}) return &p } @@ -673,13 +674,13 @@ func (p LogicalSequence) Init(ctx sessionctx.Context, offset int) *LogicalSequen // Init initializes PhysicalSequence func (p PhysicalSequence) Init(ctx sessionctx.Context, stats *property.StatsInfo, blockOffset int, props ...*property.PhysicalProperty) *PhysicalSequence { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeSequence, &p, blockOffset) - p.stats = stats + p.SetStats(stats) p.childrenReqProps = props return &p } // Init initializes ScalarSubqueryEvalCtx func (p ScalarSubqueryEvalCtx) Init(ctx sessionctx.Context, offset int) *ScalarSubqueryEvalCtx { - p.basePlan = newBasePlan(ctx, plancodec.TypeScalarSubQuery, offset) + p.Plan = base.NewBasePlan(ctx, plancodec.TypeScalarSubQuery, offset) return &p } diff --git a/planner/core/internal/base/BUILD.bazel b/planner/core/internal/base/BUILD.bazel new file mode 100644 index 0000000000000..dcee218a0bae8 --- /dev/null +++ b/planner/core/internal/base/BUILD.bazel @@ -0,0 +1,16 @@ +load("@io_bazel_rules_go//go:def.bzl", "go_library") + +go_library( + name = "base", + srcs = ["plan.go"], + importpath = "github.com/pingcap/tidb/planner/core/internal/base", + visibility = ["//planner/core:__subpackages__"], + deps = [ + "//expression", + "//planner/property", + "//sessionctx", + "//types", + "//util/stringutil", + "//util/tracing", + ], +) diff --git a/planner/core/internal/base/plan.go b/planner/core/internal/base/plan.go new file mode 100644 index 0000000000000..b644738b432c8 --- /dev/null +++ b/planner/core/internal/base/plan.go @@ -0,0 +1,138 @@ +// Copyright 2023 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +// See the License for the specific language governing permissions and +// limitations under the License. + +package base + +import ( + "fmt" + "strconv" + "unsafe" + + "github.com/pingcap/tidb/expression" + "github.com/pingcap/tidb/planner/property" + "github.com/pingcap/tidb/sessionctx" + "github.com/pingcap/tidb/types" + "github.com/pingcap/tidb/util/stringutil" + "github.com/pingcap/tidb/util/tracing" +) + +// Plan Should be used as embedded struct in Plan implementations. +type Plan struct { + ctx sessionctx.Context + stats *property.StatsInfo + tp string + id int + blockOffset int +} + +// NewBasePlan creates a new base plan. +func NewBasePlan(ctx sessionctx.Context, tp string, offset int) Plan { + id := ctx.GetSessionVars().PlanID.Add(1) + return Plan{ + tp: tp, + id: int(id), + ctx: ctx, + blockOffset: offset, + } +} + +// SCtx is to get the sessionctx from the plan. +func (p *Plan) SCtx() sessionctx.Context { + return p.ctx +} + +// SetSCtx is to set the sessionctx for the plan. +func (p *Plan) SetSCtx(ctx sessionctx.Context) { + p.ctx = ctx +} + +// OutputNames returns the outputting names of each column. +func (*Plan) OutputNames() types.NameSlice { + return nil +} + +// SetOutputNames sets the outputting name by the given slice. +func (*Plan) SetOutputNames(_ types.NameSlice) {} + +// ReplaceExprColumns implements Plan interface. +func (*Plan) ReplaceExprColumns(_ map[string]*expression.Column) {} + +// ID is to get the id. +func (p *Plan) ID() int { + return p.id +} + +// SetID is to set id. +func (p *Plan) SetID(id int) { + p.id = id +} + +// StatsInfo is to get the stats info. +func (p *Plan) StatsInfo() *property.StatsInfo { + return p.stats +} + +// ExplainInfo is to get the explain information. +func (*Plan) ExplainInfo() string { + return "N/A" +} + +// ExplainID is to get the explain ID. +func (p *Plan) ExplainID() fmt.Stringer { + return stringutil.MemoizeStr(func() string { + if p.ctx != nil && p.ctx.GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { + return p.tp + } + return p.tp + "_" + strconv.Itoa(p.id) + }) +} + +// TP is to get the tp. +func (p *Plan) TP() string { + return p.tp +} + +// SetTP is to set the tp. +func (p *Plan) SetTP(tp string) { + p.tp = tp +} + +// SelectBlockOffset is to get the select block offset. +func (p *Plan) SelectBlockOffset() int { + return p.blockOffset +} + +// SetStats sets the stats +func (p *Plan) SetStats(s *property.StatsInfo) { + p.stats = s +} + +// PlanSize is the size of BasePlan. +const PlanSize = int64(unsafe.Sizeof(Plan{})) + +// MemoryUsage return the memory usage of BasePlan +func (p *Plan) MemoryUsage() (sum int64) { + if p == nil { + return + } + + sum = PlanSize + int64(len(p.tp)) + return sum +} + +// BuildPlanTrace is to build the plan trace. +func (p *Plan) BuildPlanTrace() *tracing.PlanTrace { + planTrace := &tracing.PlanTrace{ID: p.ID(), TP: p.TP()} + return planTrace +} diff --git a/planner/core/logical_plan_builder.go b/planner/core/logical_plan_builder.go index 3b26d96e38a19..c62d41028935a 100644 --- a/planner/core/logical_plan_builder.go +++ b/planner/core/logical_plan_builder.go @@ -713,8 +713,8 @@ func (p *LogicalJoin) setPreferredJoinTypeAndOrder(hintInfo *tableHintInfo) { return } - lhsAlias := extractTableAlias(p.children[0], p.blockOffset) - rhsAlias := extractTableAlias(p.children[1], p.blockOffset) + lhsAlias := extractTableAlias(p.children[0], p.SelectBlockOffset()) + rhsAlias := extractTableAlias(p.children[1], p.SelectBlockOffset()) if hintInfo.ifPreferMergeJoin(lhsAlias) { p.preferJoinType |= preferMergeJoin p.leftPreferJoinType |= preferMergeJoin @@ -796,11 +796,11 @@ func (p *LogicalJoin) setPreferredJoinTypeAndOrder(hintInfo *tableHintInfo) { p.rightPreferJoinType |= preferHJProbe } hasConflict := false - if !p.ctx.GetSessionVars().EnableAdvancedJoinHint || p.ctx.GetSessionVars().StmtCtx.StraightJoinOrder { + if !p.SCtx().GetSessionVars().EnableAdvancedJoinHint || p.SCtx().GetSessionVars().StmtCtx.StraightJoinOrder { if containDifferentJoinTypes(p.preferJoinType) { hasConflict = true } - } else if p.ctx.GetSessionVars().EnableAdvancedJoinHint { + } else if p.SCtx().GetSessionVars().EnableAdvancedJoinHint { if containDifferentJoinTypes(p.leftPreferJoinType) || containDifferentJoinTypes(p.rightPreferJoinType) { hasConflict = true } @@ -808,7 +808,7 @@ func (p *LogicalJoin) setPreferredJoinTypeAndOrder(hintInfo *tableHintInfo) { if hasConflict { errMsg := "Join hints are conflict, you can only specify one type of join" warning := ErrInternal.GenWithStack(errMsg) - p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) p.preferJoinType = 0 } // set the join order @@ -878,7 +878,7 @@ func (p *LogicalJoin) setPreferredJoinType() { if containDifferentJoinTypes(p.preferJoinType) { errMsg := "Join hints conflict after join reorder phase, you can only specify one type of join" warning := ErrInternal.GenWithStack(errMsg) - p.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + p.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) p.preferJoinType = 0 } } @@ -905,11 +905,11 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { if ds.preferStoreType&preferTiKV == 0 { errMsg := fmt.Sprintf("No available path for table %s.%s with the store type %s of the hint /*+ read_from_storage */, "+ "please check the status of the table replica and variable value of tidb_isolation_read_engines(%v)", - ds.DBName.O, ds.table.Meta().Name.O, kv.TiKV.Name(), ds.ctx.GetSessionVars().GetIsolationReadEngines()) + ds.DBName.O, ds.table.Meta().Name.O, kv.TiKV.Name(), ds.SCtx().GetSessionVars().GetIsolationReadEngines()) warning := ErrInternal.GenWithStack(errMsg) - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + ds.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) } else { - ds.ctx.GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because you have set a hint to read table `" + hintTbl.tblName.O + "` from TiKV.") + ds.SCtx().GetSessionVars().RaiseWarningWhenMPPEnforced("MPP mode may be blocked because you have set a hint to read table `" + hintTbl.tblName.O + "` from TiKV.") } } if hintTbl := hintInfo.ifPreferTiFlash(alias); hintTbl != nil { @@ -919,7 +919,7 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { errMsg := fmt.Sprintf("Storage hints are conflict, you can only specify one storage type of table %s.%s", alias.dbName.L, alias.tblName.L) warning := ErrInternal.GenWithStack(errMsg) - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + ds.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) ds.preferStoreType = 0 return } @@ -933,9 +933,9 @@ func (ds *DataSource) setPreferredStoreType(hintInfo *tableHintInfo) { if ds.preferStoreType&preferTiFlash == 0 { errMsg := fmt.Sprintf("No available path for table %s.%s with the store type %s of the hint /*+ read_from_storage */, "+ "please check the status of the table replica and variable value of tidb_isolation_read_engines(%v)", - ds.DBName.O, ds.table.Meta().Name.O, kv.TiFlash.Name(), ds.ctx.GetSessionVars().GetIsolationReadEngines()) + ds.DBName.O, ds.table.Meta().Name.O, kv.TiFlash.Name(), ds.SCtx().GetSessionVars().GetIsolationReadEngines()) warning := ErrInternal.GenWithStack(errMsg) - ds.ctx.GetSessionVars().StmtCtx.AppendWarning(warning) + ds.SCtx().GetSessionVars().StmtCtx.AppendWarning(warning) } } } @@ -4524,7 +4524,7 @@ func (ds *DataSource) newExtraHandleSchemaCol() *expression.Column { tp.SetFlag(mysql.NotNullFlag | mysql.PriKeyFlag) return &expression.Column{ RetType: tp, - UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: ds.SCtx().GetSessionVars().AllocPlanColumnID(), ID: model.ExtraHandleID, OrigName: fmt.Sprintf("%v.%v.%v", ds.DBName, ds.tableInfo.Name, model.ExtraHandleName), } @@ -4544,7 +4544,7 @@ func (ds *DataSource) AddExtraPhysTblIDColumn() *expression.Column { } pidCol := &expression.Column{ RetType: types.NewFieldType(mysql.TypeLonglong), - UniqueID: ds.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: ds.SCtx().GetSessionVars().AllocPlanColumnID(), ID: model.ExtraPhysTblID, OrigName: fmt.Sprintf("%v.%v.%v", ds.DBName, ds.tableInfo.Name, model.ExtraPhysTblIdName), } @@ -5152,10 +5152,10 @@ func (ds *DataSource) ExtractFD() *fd.FDSet { changed bool err error ) - check := ds.ctx.GetSessionVars().IsIsolation(ast.ReadCommitted) || ds.isForUpdateRead - check = check && ds.ctx.GetSessionVars().ConnectionID > 0 + check := ds.SCtx().GetSessionVars().IsIsolation(ast.ReadCommitted) || ds.isForUpdateRead + check = check && ds.SCtx().GetSessionVars().ConnectionID > 0 if check { - latestIndexes, changed, err = getLatestIndexInfo(ds.ctx, ds.table.Meta().ID, 0) + latestIndexes, changed, err = getLatestIndexInfo(ds.SCtx(), ds.table.Meta().ID, 0) if err != nil { ds.fdSet = fds return fds @@ -5593,7 +5593,7 @@ func (b *PlanBuilder) buildSemiApply(outerPlan, innerPlan LogicalPlan, condition setIsInApplyForCTE(innerPlan, join.Schema()) ap := &LogicalApply{LogicalJoin: *join, NoDecorrelate: markNoDecorrelate} - ap.tp = plancodec.TypeApply + ap.SetTP(plancodec.TypeApply) ap.self = ap return ap, nil } diff --git a/planner/core/logical_plans.go b/planner/core/logical_plans.go index 6b170e61218b1..2bf9d24792a3a 100644 --- a/planner/core/logical_plans.go +++ b/planner/core/logical_plans.go @@ -212,7 +212,7 @@ func (p *LogicalJoin) isNAAJ() bool { // Shallow shallow copies a LogicalJoin struct. func (p *LogicalJoin) Shallow() *LogicalJoin { join := *p - return join.Init(p.ctx, p.blockOffset) + return join.Init(p.SCtx(), p.SelectBlockOffset()) } // ExtractFD implements the interface LogicalPlan. @@ -367,7 +367,7 @@ func (p *LogicalJoin) extractFDForOuterJoin(filtersFromApply []expression.Expres // if one of the inner condition is constant false, the inner side are all null, left make constant all of that. for _, one := range innerCondition { if c, ok := one.(*expression.Constant); ok && c.DeferredExpr == nil && c.ParamMarker == nil { - if isTrue, err := c.Value.ToBool(p.ctx.GetSessionVars().StmtCtx); err == nil { + if isTrue, err := c.Value.ToBool(p.SCtx().GetSessionVars().StmtCtx); err == nil { if isTrue == 0 { // c is false opt.InnerIsFalse = true @@ -858,21 +858,21 @@ func (p *LogicalProjection) ExtractFD() *fd.FDSet { // take c as constant column here. continue case *expression.Constant: - hashCode := string(x.HashCode(p.ctx.GetSessionVars().StmtCtx)) + hashCode := string(x.HashCode(p.SCtx().GetSessionVars().StmtCtx)) var ( ok bool constantUniqueID int ) if constantUniqueID, ok = fds.IsHashCodeRegistered(hashCode); !ok { constantUniqueID = outputColsUniqueIDsArray[idx] - fds.RegisterUniqueID(string(x.HashCode(p.ctx.GetSessionVars().StmtCtx)), constantUniqueID) + fds.RegisterUniqueID(string(x.HashCode(p.SCtx().GetSessionVars().StmtCtx)), constantUniqueID) } fds.AddConstants(fd.NewFastIntSet(constantUniqueID)) case *expression.ScalarFunction: // t1(a,b,c), t2(m,n) // select a, (select c+n from t2 where m=b) from t1; // expr(c+n) contains correlated column , but we can treat it as constant here. - hashCode := string(x.HashCode(p.ctx.GetSessionVars().StmtCtx)) + hashCode := string(x.HashCode(p.SCtx().GetSessionVars().StmtCtx)) var ( ok bool scalarUniqueID int @@ -905,7 +905,7 @@ func (p *LogicalProjection) ExtractFD() *fd.FDSet { // the dependent columns in scalar function should be also considered as output columns as well. outputColsUniqueIDs.Insert(int(one.UniqueID)) } - notnull := isNullRejected(p.ctx, p.schema, x) + notnull := isNullRejected(p.SCtx(), p.schema, x) if notnull || determinants.SubsetOf(fds.NotNullCols) { notnullColsUniqueIDs.Insert(scalarUniqueID) } @@ -1018,7 +1018,7 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet { // shouldn't be here, interpreted as pos param by plan builder. continue case *expression.ScalarFunction: - hashCode := string(x.HashCode(la.ctx.GetSessionVars().StmtCtx)) + hashCode := string(x.HashCode(la.SCtx().GetSessionVars().StmtCtx)) var ( ok bool scalarUniqueID int @@ -1027,8 +1027,8 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet { groupByColsUniqueIDs.Insert(scalarUniqueID) } else { // retrieve unique plan column id. 1: completely new one, allocating new unique id. 2: registered by projection earlier, using it. - if scalarUniqueID, ok = la.ctx.GetSessionVars().MapHashCode2UniqueID4ExtendedCol[hashCode]; !ok { - scalarUniqueID = int(la.ctx.GetSessionVars().AllocPlanColumnID()) + if scalarUniqueID, ok = la.SCtx().GetSessionVars().MapHashCode2UniqueID4ExtendedCol[hashCode]; !ok { + scalarUniqueID = int(la.SCtx().GetSessionVars().AllocPlanColumnID()) } fds.RegisterUniqueID(hashCode, scalarUniqueID) groupByColsUniqueIDs.Insert(scalarUniqueID) @@ -1044,7 +1044,7 @@ func (la *LogicalAggregation) ExtractFD() *fd.FDSet { determinants.Insert(int(one.UniqueID)) groupByColsOutputCols.Insert(int(one.UniqueID)) } - notnull := isNullRejected(la.ctx, la.schema, x) + notnull := isNullRejected(la.SCtx(), la.schema, x) if notnull || determinants.SubsetOf(fds.NotNullCols) { notnullColsUniqueIDs.Insert(scalarUniqueID) } @@ -1377,7 +1377,7 @@ func (la *LogicalApply) ExtractFD() *fd.FDSet { for _, col := range innerPlan.Schema().Columns { if cc.UniqueID == col.CorrelatedColUniqueID { ccc := &cc.Column - cond := expression.NewFunctionInternal(la.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), ccc, col) + cond := expression.NewFunctionInternal(la.SCtx(), ast.EQ, types.NewFieldType(mysql.TypeTiny), ccc, col) eqCond = append(eqCond, cond.(*expression.ScalarFunction)) } } @@ -1587,9 +1587,9 @@ func getTablePath(paths []*util.AccessPath) *util.AccessPath { } func (ds *DataSource) buildTableGather() LogicalPlan { - ts := LogicalTableScan{Source: ds, HandleCols: ds.handleCols}.Init(ds.ctx, ds.blockOffset) + ts := LogicalTableScan{Source: ds, HandleCols: ds.handleCols}.Init(ds.SCtx(), ds.SelectBlockOffset()) ts.SetSchema(ds.Schema()) - sg := TiKVSingleGather{Source: ds, IsIndexGather: false}.Init(ds.ctx, ds.blockOffset) + sg := TiKVSingleGather{Source: ds, IsIndexGather: false}.Init(ds.SCtx(), ds.SelectBlockOffset()) sg.SetSchema(ds.Schema()) sg.SetChildren(ts) return sg @@ -1604,7 +1604,7 @@ func (ds *DataSource) buildIndexGather(path *util.AccessPath) LogicalPlan { FullIdxColLens: path.FullIdxColLens, IdxCols: path.IdxCols, IdxColLens: path.IdxColLens, - }.Init(ds.ctx, ds.blockOffset) + }.Init(ds.SCtx(), ds.SelectBlockOffset()) is.Columns = make([]*model.ColumnInfo, len(ds.Columns)) copy(is.Columns, ds.Columns) @@ -1615,7 +1615,7 @@ func (ds *DataSource) buildIndexGather(path *util.AccessPath) LogicalPlan { Source: ds, IsIndexGather: true, Index: path.Index, - }.Init(ds.ctx, ds.blockOffset) + }.Init(ds.SCtx(), ds.SelectBlockOffset()) sg.SetSchema(ds.Schema()) sg.SetChildren(is) return sg @@ -1644,7 +1644,7 @@ func (ds *DataSource) detachCondAndBuildRangeForPath(path *util.AccessPath, cond path.TableFilters = conds return nil } - res, err := ranger.DetachCondAndBuildRangeForIndex(ds.ctx, conds, path.IdxCols, path.IdxColLens, ds.ctx.GetSessionVars().RangeMaxSize) + res, err := ranger.DetachCondAndBuildRangeForIndex(ds.SCtx(), conds, path.IdxCols, path.IdxColLens, ds.SCtx().GetSessionVars().RangeMaxSize) if err != nil { return err } @@ -1660,7 +1660,7 @@ func (ds *DataSource) detachCondAndBuildRangeForPath(path *util.AccessPath, cond path.ConstCols[i] = res.ColumnValues[i] != nil } } - path.CountAfterAccess, err = ds.tableStats.HistColl.GetRowCountByIndexRanges(ds.ctx, path.Index.ID, path.Ranges) + path.CountAfterAccess, err = ds.tableStats.HistColl.GetRowCountByIndexRanges(ds.SCtx(), path.Index.ID, path.Ranges) return err } @@ -1676,7 +1676,7 @@ func (ds *DataSource) deriveCommonHandleTablePathStats(path *util.AccessPath, co return err } if path.EqOrInCondCount == len(path.AccessConds) { - accesses, remained := path.SplitCorColAccessCondFromFilters(ds.ctx, path.EqOrInCondCount) + accesses, remained := path.SplitCorColAccessCondFromFilters(ds.SCtx(), path.EqOrInCondCount) path.AccessConds = append(path.AccessConds, accesses...) path.TableFilters = remained if len(accesses) > 0 && ds.statisticTable.Pseudo { @@ -1696,8 +1696,8 @@ func (ds *DataSource) deriveCommonHandleTablePathStats(path *util.AccessPath, co } // If the `CountAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. // We prefer the `stats.RowCount` because it could use more stats info to calculate the selectivity. - if path.CountAfterAccess < ds.stats.RowCount && !isIm { - path.CountAfterAccess = math.Min(ds.stats.RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) + if path.CountAfterAccess < ds.StatsInfo().RowCount && !isIm { + path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) } return nil } @@ -1705,9 +1705,9 @@ func (ds *DataSource) deriveCommonHandleTablePathStats(path *util.AccessPath, co // deriveTablePathStats will fulfill the information that the AccessPath need. // isIm indicates whether this function is called to generate the partial path for IndexMerge. func (ds *DataSource) deriveTablePathStats(path *util.AccessPath, conds []expression.Expression, isIm bool) error { - if ds.ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugtrace.EnterContextCommon(ds.ctx) - defer debugtrace.LeaveContextCommon(ds.ctx) + if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(ds.SCtx()) + defer debugtrace.LeaveContextCommon(ds.SCtx()) } if path.IsCommonHandlePath { return ds.deriveCommonHandleTablePathStats(path, conds, isIm) @@ -1734,7 +1734,7 @@ func (ds *DataSource) deriveTablePathStats(path *util.AccessPath, conds []expres if len(conds) == 0 { return nil } - path.AccessConds, path.TableFilters = ranger.DetachCondsForColumn(ds.ctx, conds, pkCol) + path.AccessConds, path.TableFilters = ranger.DetachCondsForColumn(ds.SCtx(), conds, pkCol) // If there's no access cond, we try to find that whether there's expression containing correlated column that // can be used to access data. corColInAccessConds := false @@ -1745,7 +1745,7 @@ func (ds *DataSource) deriveTablePathStats(path *util.AccessPath, conds []expres continue } lCol, lOk := eqFunc.GetArgs()[0].(*expression.Column) - if lOk && lCol.Equal(ds.ctx, pkCol) { + if lOk && lCol.Equal(ds.SCtx(), pkCol) { _, rOk := eqFunc.GetArgs()[1].(*expression.CorrelatedColumn) if rOk { path.AccessConds = append(path.AccessConds, filter) @@ -1755,7 +1755,7 @@ func (ds *DataSource) deriveTablePathStats(path *util.AccessPath, conds []expres } } rCol, rOk := eqFunc.GetArgs()[1].(*expression.Column) - if rOk && rCol.Equal(ds.ctx, pkCol) { + if rOk && rCol.Equal(ds.SCtx(), pkCol) { _, lOk := eqFunc.GetArgs()[0].(*expression.CorrelatedColumn) if lOk { path.AccessConds = append(path.AccessConds, filter) @@ -1771,24 +1771,24 @@ func (ds *DataSource) deriveTablePathStats(path *util.AccessPath, conds []expres return nil } var remainedConds []expression.Expression - path.Ranges, path.AccessConds, remainedConds, err = ranger.BuildTableRange(path.AccessConds, ds.ctx, pkCol.RetType, ds.ctx.GetSessionVars().RangeMaxSize) + path.Ranges, path.AccessConds, remainedConds, err = ranger.BuildTableRange(path.AccessConds, ds.SCtx(), pkCol.RetType, ds.SCtx().GetSessionVars().RangeMaxSize) path.TableFilters = append(path.TableFilters, remainedConds...) if err != nil { return err } - path.CountAfterAccess, err = ds.statisticTable.GetRowCountByIntColumnRanges(ds.ctx, pkCol.ID, path.Ranges) + path.CountAfterAccess, err = ds.statisticTable.GetRowCountByIntColumnRanges(ds.SCtx(), pkCol.ID, path.Ranges) // If the `CountAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. // We prefer the `stats.RowCount` because it could use more stats info to calculate the selectivity. - if path.CountAfterAccess < ds.stats.RowCount && !isIm { - path.CountAfterAccess = math.Min(ds.stats.RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) + if path.CountAfterAccess < ds.StatsInfo().RowCount && !isIm { + path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) } return err } func (ds *DataSource) fillIndexPath(path *util.AccessPath, conds []expression.Expression) error { - if ds.ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugtrace.EnterContextCommon(ds.ctx) - defer debugtrace.LeaveContextCommon(ds.ctx) + if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(ds.SCtx()) + defer debugtrace.LeaveContextCommon(ds.SCtx()) } path.Ranges = ranger.FullRange() path.CountAfterAccess = float64(ds.statisticTable.RealtimeCount) @@ -1822,12 +1822,12 @@ func (ds *DataSource) fillIndexPath(path *util.AccessPath, conds []expression.Ex // conds is the conditions used to generate the DetachRangeResult for path. // isIm indicates whether this function is called to generate the partial path for IndexMerge. func (ds *DataSource) deriveIndexPathStats(path *util.AccessPath, _ []expression.Expression, isIm bool) { - if ds.ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugtrace.EnterContextCommon(ds.ctx) - defer debugtrace.LeaveContextCommon(ds.ctx) + if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(ds.SCtx()) + defer debugtrace.LeaveContextCommon(ds.SCtx()) } if path.EqOrInCondCount == len(path.AccessConds) { - accesses, remained := path.SplitCorColAccessCondFromFilters(ds.ctx, path.EqOrInCondCount) + accesses, remained := path.SplitCorColAccessCondFromFilters(ds.SCtx(), path.EqOrInCondCount) path.AccessConds = append(path.AccessConds, accesses...) path.TableFilters = remained if len(accesses) > 0 && ds.statisticTable.Pseudo { @@ -1850,11 +1850,11 @@ func (ds *DataSource) deriveIndexPathStats(path *util.AccessPath, _ []expression path.IndexFilters = append(path.IndexFilters, indexFilters...) // If the `CountAfterAccess` is less than `stats.RowCount`, there must be some inconsistent stats info. // We prefer the `stats.RowCount` because it could use more stats info to calculate the selectivity. - if path.CountAfterAccess < ds.stats.RowCount && !isIm { - path.CountAfterAccess = math.Min(ds.stats.RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) + if path.CountAfterAccess < ds.StatsInfo().RowCount && !isIm { + path.CountAfterAccess = math.Min(ds.StatsInfo().RowCount/SelectionFactor, float64(ds.statisticTable.RealtimeCount)) } if path.IndexFilters != nil { - selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.ctx, path.IndexFilters, nil) + selectivity, _, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), path.IndexFilters, nil) if err != nil { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor @@ -1862,7 +1862,7 @@ func (ds *DataSource) deriveIndexPathStats(path *util.AccessPath, _ []expression if isIm { path.CountAfterIndex = path.CountAfterAccess * selectivity } else { - path.CountAfterIndex = math.Max(path.CountAfterAccess*selectivity, ds.stats.RowCount) + path.CountAfterIndex = math.Max(path.CountAfterAccess*selectivity, ds.StatsInfo().RowCount) } } else { path.CountAfterIndex = path.CountAfterAccess diff --git a/planner/core/logical_plans_test.go b/planner/core/logical_plans_test.go index 5ff44ab9e5a3c..164ed95f2dae8 100644 --- a/planner/core/logical_plans_test.go +++ b/planner/core/logical_plans_test.go @@ -701,7 +701,7 @@ func TestAllocID(t *testing.T) { ctx := MockContext() pA := DataSource{}.Init(ctx, 0) pB := DataSource{}.Init(ctx, 0) - require.Equal(t, pB.id, pA.id+1) + require.Equal(t, pB.ID(), pA.ID()+1) } func checkDataSourceCols(p LogicalPlan, t *testing.T, ans map[int][]string, comment string) { diff --git a/planner/core/memtable_predicate_extractor.go b/planner/core/memtable_predicate_extractor.go index dc859e8660a1a..49c82f991e53d 100644 --- a/planner/core/memtable_predicate_extractor.go +++ b/planner/core/memtable_predicate_extractor.go @@ -711,11 +711,11 @@ func (e *ClusterLogTableExtractor) explainInfo(p *PhysicalMemTable) string { st, et := e.StartTime, e.EndTime if st > 0 { st := time.UnixMilli(st) - fmt.Fprintf(r, "start_time:%v, ", st.In(p.ctx.GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat)) + fmt.Fprintf(r, "start_time:%v, ", st.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat)) } if et > 0 { et := time.UnixMilli(et) - fmt.Fprintf(r, "end_time:%v, ", et.In(p.ctx.GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat)) + fmt.Fprintf(r, "end_time:%v, ", et.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat)) } if len(e.NodeTypes) > 0 { fmt.Fprintf(r, "node_types:[%s], ", extractStringFromStringSet(e.NodeTypes)) @@ -846,11 +846,11 @@ func (e *HotRegionsHistoryTableExtractor) explainInfo(p *PhysicalMemTable) strin st, et := e.StartTime, e.EndTime if st > 0 { st := time.UnixMilli(st) - fmt.Fprintf(r, "start_time:%v, ", st.In(p.ctx.GetSessionVars().StmtCtx.TimeZone).Format(time.DateTime)) + fmt.Fprintf(r, "start_time:%v, ", st.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone).Format(time.DateTime)) } if et > 0 { et := time.UnixMilli(et) - fmt.Fprintf(r, "end_time:%v, ", et.In(p.ctx.GetSessionVars().StmtCtx.TimeZone).Format(time.DateTime)) + fmt.Fprintf(r, "end_time:%v, ", et.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone).Format(time.DateTime)) } if len(e.RegionIDs) > 0 { fmt.Fprintf(r, "region_ids:[%s], ", extractStringFromUint64Slice(e.RegionIDs)) @@ -958,13 +958,13 @@ func (e *MetricTableExtractor) explainInfo(p *PhysicalMemTable) string { if e.SkipRequest { return "skip_request: true" } - promQL := e.GetMetricTablePromQL(p.ctx, p.Table.Name.L) + promQL := e.GetMetricTablePromQL(p.SCtx(), p.Table.Name.L) startTime, endTime := e.StartTime, e.EndTime - step := time.Second * time.Duration(p.ctx.GetSessionVars().MetricSchemaStep) + step := time.Second * time.Duration(p.SCtx().GetSessionVars().MetricSchemaStep) return fmt.Sprintf("PromQL:%v, start_time:%v, end_time:%v, step:%v", promQL, - startTime.In(p.ctx.GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat), - endTime.In(p.ctx.GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat), + startTime.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat), + endTime.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone).Format(MetricTableTimeFormat), step, ) } @@ -1321,10 +1321,10 @@ func (e *SlowQueryExtractor) explainInfo(p *PhysicalMemTable) string { return "skip_request: true" } if !e.Enable { - return fmt.Sprintf("only search in the current '%v' file", p.ctx.GetSessionVars().SlowQueryFile) + return fmt.Sprintf("only search in the current '%v' file", p.SCtx().GetSessionVars().SlowQueryFile) } - startTime := e.TimeRanges[0].StartTime.In(p.ctx.GetSessionVars().StmtCtx.TimeZone) - endTime := e.TimeRanges[0].EndTime.In(p.ctx.GetSessionVars().StmtCtx.TimeZone) + startTime := e.TimeRanges[0].StartTime.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone) + endTime := e.TimeRanges[0].EndTime.In(p.SCtx().GetSessionVars().StmtCtx.TimeZone) return fmt.Sprintf("start_time:%v, end_time:%v", types.NewTime(types.FromGoTime(startTime), mysql.TypeDatetime, types.MaxFsp).String(), types.NewTime(types.FromGoTime(endTime), mysql.TypeDatetime, types.MaxFsp).String()) @@ -1449,8 +1449,8 @@ func (e *StatementsSummaryExtractor) explainInfo(p *PhysicalMemTable) string { if !e.Digests.Empty() { fmt.Fprintf(buf, "digests: [%s], ", extractStringFromStringSet(e.Digests)) } - if e.CoarseTimeRange != nil && p.ctx.GetSessionVars() != nil && p.ctx.GetSessionVars().StmtCtx != nil { - stmtCtx := p.ctx.GetSessionVars().StmtCtx + if e.CoarseTimeRange != nil && p.SCtx().GetSessionVars() != nil && p.SCtx().GetSessionVars().StmtCtx != nil { + stmtCtx := p.SCtx().GetSessionVars().StmtCtx startTime := e.CoarseTimeRange.StartTime.In(stmtCtx.TimeZone) endTime := e.CoarseTimeRange.EndTime.In(stmtCtx.TimeZone) startTimeStr := types.NewTime(types.FromGoTime(startTime), mysql.TypeDatetime, types.MaxFsp).String() diff --git a/planner/core/optimizer.go b/planner/core/optimizer.go index d37355df76a43..bf2d055c9b4bc 100644 --- a/planner/core/optimizer.go +++ b/planner/core/optimizer.go @@ -132,7 +132,7 @@ func (op *logicalOptimizeOp) appendBeforeRuleOptimize(index int, name string, be if op == nil || op.tracer == nil { return } - op.tracer.AppendRuleTracerBeforeRuleOptimize(index, name, before.buildPlanTrace()) + op.tracer.AppendRuleTracerBeforeRuleOptimize(index, name, before.BuildPlanTrace()) } func (op *logicalOptimizeOp) appendStepToCurrent(id int, tp string, reason, action func() string) { @@ -146,7 +146,7 @@ func (op *logicalOptimizeOp) recordFinalLogicalPlan(final LogicalPlan) { if op == nil || op.tracer == nil { return } - op.tracer.RecordFinalLogicalPlan(final.buildPlanTrace()) + op.tracer.RecordFinalLogicalPlan(final.BuildPlanTrace()) } // logicalOptRule means a logical optimizing rule, which contains decorrelate, ppd, column pruning, etc. @@ -324,7 +324,7 @@ func DoOptimizeAndLogicAsRet(ctx context.Context, sctx sessionctx.Context, flag refineCETrace(sctx) } if sessVars.StmtCtx.EnableOptimizeTrace { - sessVars.StmtCtx.OptimizeTracer.RecordFinalPlan(finalPlan.buildPlanTrace()) + sessVars.StmtCtx.OptimizeTracer.RecordFinalPlan(finalPlan.BuildPlanTrace()) } return logic, finalPlan, cost, nil } @@ -515,7 +515,7 @@ func prunePhysicalColumnForHashJoinChild(sctx sessionctx.Context, hashJoin *Phys ch := sender.children[0] proj := PhysicalProjection{ Exprs: usedExprs, - }.Init(sctx, ch.statsInfo(), ch.SelectBlockOffset()) + }.Init(sctx, ch.StatsInfo(), ch.SelectBlockOffset()) proj.SetSchema(prunedSchema) proj.SetChildren(ch) @@ -689,7 +689,7 @@ func rewriteTableScanAndAggArgs(physicalTableScan *PhysicalTableScan, aggFuncs [ if columnInfo.GetFlen() < resultColumnInfo.GetFlen() { resultColumnInfo = columnInfo resultColumn = &expression.Column{ - UniqueID: physicalTableScan.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: physicalTableScan.SCtx().GetSessionVars().AllocPlanColumnID(), ID: resultColumnInfo.ID, RetType: resultColumnInfo.FieldType.Clone(), OrigName: fmt.Sprintf("%s.%s.%s", physicalTableScan.DBName.L, physicalTableScan.Table.Name.L, resultColumnInfo.Name), @@ -1179,7 +1179,7 @@ func physicalOptimize(logic LogicalPlan, planCounter *PlanCounterTp) (plan Physi panic(r) /* pass panic to upper function to handle */ } if err == nil { - tracer.RecordFinalPlanTrace(plan.buildPlanTrace()) + tracer.RecordFinalPlanTrace(plan.BuildPlanTrace()) stmtCtx.OptimizeTracer.Physical = tracer } }() diff --git a/planner/core/physical_plans.go b/planner/core/physical_plans.go index d9adc80bd87c7..83dbe973e3363 100644 --- a/planner/core/physical_plans.go +++ b/planner/core/physical_plans.go @@ -200,7 +200,7 @@ func (p *PhysicalTableReader) GetTableScan() (*PhysicalTableScan, error) { // GetAvgRowSize return the average row size of this plan. func (p *PhysicalTableReader) GetAvgRowSize() float64 { - return getTblStats(p.tablePlan).GetAvgRowSize(p.ctx, p.tablePlan.Schema().Columns, false, false) + return getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false) } // MemoryUsage return the memory usage of PhysicalTableReader @@ -233,14 +233,14 @@ func setMppOrBatchCopForTableScan(curPlan PhysicalPlan) { // GetPhysicalTableReader returns PhysicalTableReader for logical TiKVSingleGather. func (sg *TiKVSingleGather) GetPhysicalTableReader(schema *expression.Schema, stats *property.StatsInfo, props ...*property.PhysicalProperty) *PhysicalTableReader { - reader := PhysicalTableReader{}.Init(sg.ctx, sg.blockOffset) + reader := PhysicalTableReader{}.Init(sg.SCtx(), sg.SelectBlockOffset()) reader.PartitionInfo = PartitionInfo{ PruningConds: sg.Source.allConds, PartitionNames: sg.Source.partitionNames, Columns: sg.Source.TblCols, ColumnNames: sg.Source.names, } - reader.stats = stats + reader.SetStats(stats) reader.SetSchema(schema) reader.childrenReqProps = props return reader @@ -248,8 +248,8 @@ func (sg *TiKVSingleGather) GetPhysicalTableReader(schema *expression.Schema, st // GetPhysicalIndexReader returns PhysicalIndexReader for logical TiKVSingleGather. func (sg *TiKVSingleGather) GetPhysicalIndexReader(schema *expression.Schema, stats *property.StatsInfo, props ...*property.PhysicalProperty) *PhysicalIndexReader { - reader := PhysicalIndexReader{}.Init(sg.ctx, sg.blockOffset) - reader.stats = stats + reader := PhysicalIndexReader{}.Init(sg.SCtx(), sg.SelectBlockOffset()) + reader.SetStats(stats) reader.SetSchema(schema) reader.childrenReqProps = props return reader @@ -288,10 +288,11 @@ func (p *PhysicalTableReader) ExtractCorrelatedCols() (corCols []*expression.Cor return corCols } -func (p *PhysicalTableReader) buildPlanTrace() *tracing.PlanTrace { - rp := p.basePhysicalPlan.buildPlanTrace() +// BuildPlanTrace implements PhysicalPlan interface. +func (p *PhysicalTableReader) BuildPlanTrace() *tracing.PlanTrace { + rp := p.basePhysicalPlan.BuildPlanTrace() if p.tablePlan != nil { - rp.Children = append(rp.Children, p.tablePlan.buildPlanTrace()) + rp.Children = append(rp.Children, p.tablePlan.BuildPlanTrace()) } return rp } @@ -363,10 +364,11 @@ func (p *PhysicalIndexReader) ExtractCorrelatedCols() (corCols []*expression.Cor return corCols } -func (p *PhysicalIndexReader) buildPlanTrace() *tracing.PlanTrace { - rp := p.basePhysicalPlan.buildPlanTrace() +// BuildPlanTrace implements PhysicalPlan interface. +func (p *PhysicalIndexReader) BuildPlanTrace() *tracing.PlanTrace { + rp := p.basePhysicalPlan.BuildPlanTrace() if p.indexPlan != nil { - rp.Children = append(rp.Children, p.indexPlan.buildPlanTrace()) + rp.Children = append(rp.Children, p.indexPlan.BuildPlanTrace()) } return rp } @@ -490,21 +492,22 @@ func (p *PhysicalIndexLookUpReader) ExtractCorrelatedCols() (corCols []*expressi // GetIndexNetDataSize return the estimated total size in bytes via network transfer. func (p *PhysicalIndexLookUpReader) GetIndexNetDataSize() float64 { - return getTblStats(p.indexPlan).GetAvgRowSize(p.ctx, p.indexPlan.Schema().Columns, true, false) * p.indexPlan.StatsCount() + return getTblStats(p.indexPlan).GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false) * p.indexPlan.StatsCount() } // GetAvgTableRowSize return the average row size of each final row. func (p *PhysicalIndexLookUpReader) GetAvgTableRowSize() float64 { - return getTblStats(p.tablePlan).GetAvgRowSize(p.ctx, p.tablePlan.Schema().Columns, false, false) + return getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false) } -func (p *PhysicalIndexLookUpReader) buildPlanTrace() *tracing.PlanTrace { - rp := p.basePhysicalPlan.buildPlanTrace() +// BuildPlanTrace implements PhysicalPlan interface. +func (p *PhysicalIndexLookUpReader) BuildPlanTrace() *tracing.PlanTrace { + rp := p.basePhysicalPlan.BuildPlanTrace() if p.indexPlan != nil { - rp.Children = append(rp.Children, p.indexPlan.buildPlanTrace()) + rp.Children = append(rp.Children, p.indexPlan.BuildPlanTrace()) } if p.tablePlan != nil { - rp.Children = append(rp.Children, p.tablePlan.buildPlanTrace()) + rp.Children = append(rp.Children, p.tablePlan.BuildPlanTrace()) } return rp } @@ -598,13 +601,14 @@ func (p *PhysicalIndexMergeReader) ExtractCorrelatedCols() (corCols []*expressio return corCols } -func (p *PhysicalIndexMergeReader) buildPlanTrace() *tracing.PlanTrace { - rp := p.basePhysicalPlan.buildPlanTrace() +// BuildPlanTrace implements PhysicalPlan interface. +func (p *PhysicalIndexMergeReader) BuildPlanTrace() *tracing.PlanTrace { + rp := p.basePhysicalPlan.BuildPlanTrace() if p.tablePlan != nil { - rp.Children = append(rp.Children, p.tablePlan.buildPlanTrace()) + rp.Children = append(rp.Children, p.tablePlan.BuildPlanTrace()) } for _, partialPlan := range p.partialPlans { - rp.Children = append(rp.Children, partialPlan.buildPlanTrace()) + rp.Children = append(rp.Children, partialPlan.BuildPlanTrace()) } return rp } @@ -1426,9 +1430,9 @@ func NewPhysicalHashJoin(p *LogicalJoin, innerIdx int, useOuterToBuild bool, new basePhysicalJoin: baseJoin, EqualConditions: p.EqualConditions, NAEqualConditions: p.NAEQConditions, - Concurrency: uint(p.ctx.GetSessionVars().HashJoinConcurrency()), + Concurrency: uint(p.SCtx().GetSessionVars().HashJoinConcurrency()), UseOuterToBuild: useOuterToBuild, - }.Init(p.ctx, newStats, p.blockOffset, prop...) + }.Init(p.SCtx(), newStats, p.SelectBlockOffset(), prop...) return hashJoin } @@ -1610,7 +1614,7 @@ type PhysicalExpand struct { func (p PhysicalExpand) Init(ctx sessionctx.Context, stats *property.StatsInfo, offset int, props ...*property.PhysicalProperty) *PhysicalExpand { p.basePhysicalPlan = newBasePhysicalPlan(ctx, plancodec.TypeExpand, &p, offset) p.childrenReqProps = props - p.stats = stats + p.SetStats(stats) return &p } @@ -1982,7 +1986,7 @@ func NewPhysicalHashAgg(la *LogicalAggregation, newStats *property.StatsInfo, pr agg := basePhysicalAgg{ GroupByItems: newGbyItems, AggFuncs: newAggFuncs, - }.initForHash(la.ctx, newStats, la.blockOffset, prop) + }.initForHash(la.SCtx(), newStats, la.SelectBlockOffset(), prop) return agg } @@ -2428,9 +2432,9 @@ func CollectPlanStatsVersion(plan PhysicalPlan, statsInfos map[string]uint64) ma // because they use the same stats and we do not set the stats info for tablePlan. statsInfos = CollectPlanStatsVersion(copPlan.indexPlan, statsInfos) case *PhysicalIndexScan: - statsInfos[copPlan.Table.Name.O] = copPlan.stats.StatsVersion + statsInfos[copPlan.Table.Name.O] = copPlan.StatsInfo().StatsVersion case *PhysicalTableScan: - statsInfos[copPlan.Table.Name.O] = copPlan.stats.StatsVersion + statsInfos[copPlan.Table.Name.O] = copPlan.StatsInfo().StatsVersion } return statsInfos @@ -2578,10 +2582,10 @@ func (p *PhysicalCTE) ExplainInfo() string { // ExplainID overrides the ExplainID. func (p *PhysicalCTE) ExplainID() fmt.Stringer { return stringutil.MemoizeStr(func() string { - if p.ctx != nil && p.ctx.GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { + if p.SCtx() != nil && p.SCtx().GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { return p.TP() } - return p.TP() + "_" + strconv.Itoa(p.id) + return p.TP() + "_" + strconv.Itoa(p.ID()) }) } @@ -2770,10 +2774,10 @@ func (p *PhysicalSequence) MemoryUsage() (sum int64) { // ExplainID overrides the ExplainID. func (p *PhysicalSequence) ExplainID() fmt.Stringer { return stringutil.MemoizeStr(func() string { - if p.ctx != nil && p.ctx.GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { + if p.SCtx() != nil && p.SCtx().GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { return p.TP() } - return p.TP() + "_" + strconv.Itoa(p.id) + return p.TP() + "_" + strconv.Itoa(p.ID()) }) } diff --git a/planner/core/plan.go b/planner/core/plan.go index 586eb8db1588a..942c05b772043 100644 --- a/planner/core/plan.go +++ b/planner/core/plan.go @@ -17,12 +17,11 @@ package core import ( "fmt" "math" - "strconv" - "unsafe" "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/kv" + "github.com/pingcap/tidb/planner/core/internal/base" fd "github.com/pingcap/tidb/planner/funcdep" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" @@ -31,7 +30,6 @@ import ( "github.com/pingcap/tidb/util/execdetails" "github.com/pingcap/tidb/util/mathutil" "github.com/pingcap/tidb/util/size" - "github.com/pingcap/tidb/util/stringutil" "github.com/pingcap/tidb/util/tracing" "github.com/pingcap/tipb/go-tipb" ) @@ -55,13 +53,13 @@ type Plan interface { // ExplainInfo returns operator information to be explained. ExplainInfo() string - // replaceExprColumns replace all the column reference in the plan's expression node. - replaceExprColumns(replace map[string]*expression.Column) + // ReplaceExprColumns replace all the column reference in the plan's expression node. + ReplaceExprColumns(replace map[string]*expression.Column) SCtx() sessionctx.Context - // statsInfo will return the property.StatsInfo for this plan. - statsInfo() *property.StatsInfo + // StatsInfo will return the property.StatsInfo for this plan. + StatsInfo() *property.StatsInfo // OutputNames returns the outputting names of each column. OutputNames() types.NameSlice @@ -71,7 +69,7 @@ type Plan interface { SelectBlockOffset() int - buildPlanTrace() *tracing.PlanTrace + BuildPlanTrace() *tracing.PlanTrace } func enforceProperty(p *property.PhysicalProperty, tsk task, ctx sessionctx.Context) task { @@ -96,7 +94,7 @@ func enforceProperty(p *property.PhysicalProperty, tsk task, ctx sessionctx.Cont sort := PhysicalSort{ ByItems: make([]*util.ByItems, 0, len(p.SortItems)), IsPartialSort: p.IsSortItemAllForPartition(), - }.Init(ctx, tsk.plan().statsInfo(), tsk.plan().SelectBlockOffset(), sortReqProp) + }.Init(ctx, tsk.plan().StatsInfo(), tsk.plan().SelectBlockOffset(), sortReqProp) for _, col := range p.SortItems { sort.ByItems = append(sort.ByItems, &util.ByItems{Expr: col.Col, Desc: col.Desc}) } @@ -144,7 +142,7 @@ func optimizeByShuffle4Window(pp *PhysicalWindow, ctx sessionctx.Context) *Physi for _, item := range pp.PartitionBy { partitionBy = append(partitionBy, item.Col) } - ndv, _ := getColsNDVWithMatchedLen(partitionBy, dataSource.Schema(), dataSource.statsInfo()) + ndv, _ := getColsNDVWithMatchedLen(partitionBy, dataSource.Schema(), dataSource.StatsInfo()) if ndv <= 1 { return nil } @@ -161,7 +159,7 @@ func optimizeByShuffle4Window(pp *PhysicalWindow, ctx sessionctx.Context) *Physi DataSources: []PhysicalPlan{dataSource}, SplitterType: PartitionHashSplitterType, ByItemArrays: [][]expression.Expression{byItems}, - }.Init(ctx, pp.statsInfo(), pp.SelectBlockOffset(), reqProp) + }.Init(ctx, pp.StatsInfo(), pp.SelectBlockOffset(), reqProp) return shuffle } @@ -185,7 +183,7 @@ func optimizeByShuffle4StreamAgg(pp *PhysicalStreamAgg, ctx sessionctx.Context) partitionBy = append(partitionBy, col) } } - ndv, _ := getColsNDVWithMatchedLen(partitionBy, dataSource.Schema(), dataSource.statsInfo()) + ndv, _ := getColsNDVWithMatchedLen(partitionBy, dataSource.Schema(), dataSource.StatsInfo()) if ndv <= 1 { return nil } @@ -198,7 +196,7 @@ func optimizeByShuffle4StreamAgg(pp *PhysicalStreamAgg, ctx sessionctx.Context) DataSources: []PhysicalPlan{dataSource}, SplitterType: PartitionHashSplitterType, ByItemArrays: [][]expression.Expression{util.CloneExprs(pp.GroupByItems)}, - }.Init(ctx, pp.statsInfo(), pp.SelectBlockOffset(), reqProp) + }.Init(ctx, pp.StatsInfo(), pp.SelectBlockOffset(), reqProp) return shuffle } @@ -237,7 +235,7 @@ func optimizeByShuffle4MergeJoin(pp *PhysicalMergeJoin, ctx sessionctx.Context) DataSources: dataSources, SplitterType: PartitionHashSplitterType, ByItemArrays: [][]expression.Expression{leftByItemArray, rightByItemArray}, - }.Init(ctx, pp.statsInfo(), pp.SelectBlockOffset(), reqProp) + }.Init(ctx, pp.StatsInfo(), pp.SelectBlockOffset(), reqProp) return shuffle } @@ -371,8 +369,8 @@ type PhysicalPlan interface { // ResolveIndices resolves the indices for columns. After doing this, the columns can evaluate the rows by their indices. ResolveIndices() error - // Stats returns the StatsInfo of the plan. - Stats() *property.StatsInfo + // StatsInfo returns the StatsInfo of the plan. + StatsInfo() *property.StatsInfo // SetStats sets basePlan.stats inside the basePhysicalPlan. SetStats(s *property.StatsInfo) @@ -390,9 +388,9 @@ type PhysicalPlan interface { // MemoryUsage return the memory usage of PhysicalPlan MemoryUsage() int64 - // Below three methods help to handle the inconsistency between row count in the statsInfo and the recorded + // Below three methods help to handle the inconsistency between row count in the StatsInfo and the recorded // actual row count. - // For the children in the inner side (probe side) of Index Join and Apply, the row count in the statsInfo + // For the children in the inner side (probe side) of Index Join and Apply, the row count in the StatsInfo // means the estimated row count for a single "probe", but the recorded actual row count is the total row // count for all "probes". // To handle this inconsistency without breaking anything else, we added a field `probeParents` of @@ -403,7 +401,7 @@ type PhysicalPlan interface { // setProbeParents sets the above stated `probeParents` field. setProbeParents([]PhysicalPlan) - // getEstRowCountForDisplay uses the "single probe" row count in statsInfo and the probeParents to calculate + // getEstRowCountForDisplay uses the "single probe" row count in StatsInfo and the probeParents to calculate // the "all probe" row count. // All places that display the row count for a PhysicalPlan are expected to use this method. getEstRowCountForDisplay() float64 @@ -444,7 +442,7 @@ func (op *PlanCostOption) WithOptimizeTracer(v *physicalOptimizeOp) *PlanCostOpt } type baseLogicalPlan struct { - basePlan + base.Plan taskMap map[string]task // taskMapBak forms a backlog stack of taskMap, used to roll back the taskMap. @@ -489,7 +487,7 @@ func getEstimatedProbeCntFromProbeParents(probeParents []PhysicalPlan) float64 { case *PhysicalApply, *PhysicalIndexHashJoin, *PhysicalIndexMergeJoin, *PhysicalIndexJoin: if join, ok := pp.(interface{ getInnerChildIdx() int }); ok { outer := pp.Children()[1-join.getInnerChildIdx()] - res *= outer.statsInfo().RowCount + res *= outer.StatsInfo().RowCount } } } @@ -519,7 +517,7 @@ func getActualProbeCntFromProbeParents(pps []PhysicalPlan, statsColl *execdetail } type basePhysicalPlan struct { - basePlan + base.Plan childrenReqProps []*property.PhysicalProperty self PhysicalPlan @@ -542,7 +540,7 @@ type basePhysicalPlan struct { func (p *basePhysicalPlan) cloneWithSelf(newSelf PhysicalPlan) (*basePhysicalPlan, error) { base := &basePhysicalPlan{ - basePlan: p.basePlan, + Plan: p.Plan, self: newSelf, TiFlashFineGrainedShuffleStreamCount: p.TiFlashFineGrainedShuffleStreamCount, probeParents: p.probeParents, @@ -593,7 +591,7 @@ func (p *basePhysicalPlan) MemoryUsage() (sum int64) { return } - sum = p.basePlan.MemoryUsage() + size.SizeOfSlice + int64(cap(p.childrenReqProps))*size.SizeOfPointer + + sum = p.Plan.MemoryUsage() + size.SizeOfSlice + int64(cap(p.childrenReqProps))*size.SizeOfPointer + size.SizeOfSlice + int64(cap(p.children)+1)*size.SizeOfInterface + size.SizeOfFloat64 + size.SizeOfUint64 + size.SizeOfBool @@ -610,7 +608,7 @@ func (p *basePhysicalPlan) getEstRowCountForDisplay() float64 { if p == nil { return 0 } - return p.statsInfo().RowCount * getEstimatedProbeCntFromProbeParents(p.probeParents) + return p.StatsInfo().RowCount * getEstimatedProbeCntFromProbeParents(p.probeParents) } func (p *basePhysicalPlan) getActualProbeCnt(statsColl *execdetails.RuntimeStatsColl) int64 { @@ -626,12 +624,12 @@ func (p *basePhysicalPlan) setProbeParents(probeParents []PhysicalPlan) { // GetLogicalTS4TaskMap get the logical TimeStamp now to help rollback the TaskMap changes after that. func (p *baseLogicalPlan) GetLogicalTS4TaskMap() uint64 { - p.ctx.GetSessionVars().StmtCtx.TaskMapBakTS++ - return p.ctx.GetSessionVars().StmtCtx.TaskMapBakTS + p.SCtx().GetSessionVars().StmtCtx.TaskMapBakTS++ + return p.SCtx().GetSessionVars().StmtCtx.TaskMapBakTS } func (p *baseLogicalPlan) rollBackTaskMap(ts uint64) { - if !p.ctx.GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { + if !p.SCtx().GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { return } if len(p.taskMapBak) > 0 { @@ -665,7 +663,7 @@ func (p *baseLogicalPlan) getTask(prop *property.PhysicalProperty) task { func (p *baseLogicalPlan) storeTask(prop *property.PhysicalProperty, task task) { key := prop.HashCode() - if p.ctx.GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { + if p.SCtx().GetSessionVars().StmtCtx.StmtHints.TaskMapNeedBackUp() { // Empty string for useless change. ts := p.GetLogicalTS4TaskMap() p.taskMapBakTS = append(p.taskMapBakTS, ts) @@ -730,30 +728,20 @@ func (p *logicalSchemaProducer) BuildKeyInfo(selfSchema *expression.Schema, chil } } -func newBasePlan(ctx sessionctx.Context, tp string, offset int) basePlan { - id := ctx.GetSessionVars().PlanID.Add(1) - return basePlan{ - tp: tp, - id: int(id), - ctx: ctx, - blockOffset: offset, - } -} - func newBaseLogicalPlan(ctx sessionctx.Context, tp string, self LogicalPlan, offset int) baseLogicalPlan { return baseLogicalPlan{ taskMap: make(map[string]task), taskMapBak: make([]string, 0, 10), taskMapBakTS: make([]uint64, 0, 10), - basePlan: newBasePlan(ctx, tp, offset), + Plan: base.NewBasePlan(ctx, tp, offset), self: self, } } func newBasePhysicalPlan(ctx sessionctx.Context, tp string, self PhysicalPlan, offset int) basePhysicalPlan { return basePhysicalPlan{ - basePlan: newBasePlan(ctx, tp, offset), - self: self, + Plan: base.NewBasePlan(ctx, tp, offset), + self: self, } } @@ -769,81 +757,6 @@ func (p *baseLogicalPlan) PruneColumns(parentUsedCols []*expression.Column, opt return p.children[0].PruneColumns(parentUsedCols, opt) } -// basePlan implements base Plan interface. -// Should be used as embedded struct in Plan implementations. -type basePlan struct { - tp string - id int - ctx sessionctx.Context - stats *property.StatsInfo - blockOffset int -} - -// OutputNames returns the outputting names of each column. -func (*basePlan) OutputNames() types.NameSlice { - return nil -} - -func (*basePlan) SetOutputNames(_ types.NameSlice) {} - -func (*basePlan) replaceExprColumns(_ map[string]*expression.Column) {} - -// ID implements Plan ID interface. -func (p *basePlan) ID() int { - return p.id -} - -// property.StatsInfo implements the Plan interface. -func (p *basePlan) statsInfo() *property.StatsInfo { - return p.stats -} - -// ExplainInfo implements Plan interface. -func (*basePlan) ExplainInfo() string { - return "N/A" -} - -func (p *basePlan) ExplainID() fmt.Stringer { - return stringutil.MemoizeStr(func() string { - if p.ctx != nil && p.ctx.GetSessionVars().StmtCtx.IgnoreExplainIDSuffix { - return p.tp - } - return p.tp + "_" + strconv.Itoa(p.id) - }) -} - -// TP implements Plan interface. -func (p *basePlan) TP() string { - return p.tp -} - -func (p *basePlan) SelectBlockOffset() int { - return p.blockOffset -} - -// Stats implements Plan Stats interface. -func (p *basePlan) Stats() *property.StatsInfo { - return p.stats -} - -// SetStats sets basePlan.stats -func (p *basePlan) SetStats(s *property.StatsInfo) { - p.stats = s -} - -// basePlanSize is the size of basePlan. -const basePlanSize = int64(unsafe.Sizeof(basePlan{})) - -// MemoryUsage return the memory usage of basePlan -func (p *basePlan) MemoryUsage() (sum int64) { - if p == nil { - return - } - - sum = basePlanSize + int64(len(p.tp)) - return sum -} - // Schema implements Plan Schema interface. func (p *baseLogicalPlan) Schema() *expression.Schema { return p.children[0].Schema() @@ -892,13 +805,8 @@ func (p *basePhysicalPlan) SetChild(i int, child PhysicalPlan) { p.children[i] = child } -// Context implements Plan Context interface. -func (p *basePlan) SCtx() sessionctx.Context { - return p.ctx -} - -// buildPlanTrace implements Plan -func (p *basePhysicalPlan) buildPlanTrace() *tracing.PlanTrace { +// BuildPlanTrace implements Plan +func (p *basePhysicalPlan) BuildPlanTrace() *tracing.PlanTrace { tp := "" info := "" if p.self != nil { @@ -908,26 +816,20 @@ func (p *basePhysicalPlan) buildPlanTrace() *tracing.PlanTrace { planTrace := &tracing.PlanTrace{ID: p.ID(), TP: tp, ExplainInfo: info} for _, child := range p.Children() { - planTrace.Children = append(planTrace.Children, child.buildPlanTrace()) + planTrace.Children = append(planTrace.Children, child.BuildPlanTrace()) } return planTrace } -// buildPlanTrace implements Plan -func (p *baseLogicalPlan) buildPlanTrace() *tracing.PlanTrace { +// BuildPlanTrace implements Plan +func (p *baseLogicalPlan) BuildPlanTrace() *tracing.PlanTrace { planTrace := &tracing.PlanTrace{ID: p.ID(), TP: p.TP(), ExplainInfo: p.self.ExplainInfo()} for _, child := range p.Children() { - planTrace.Children = append(planTrace.Children, child.buildPlanTrace()) + planTrace.Children = append(planTrace.Children, child.BuildPlanTrace()) } return planTrace } -// buildPlanTrace implements Plan -func (p *basePlan) buildPlanTrace() *tracing.PlanTrace { - planTrace := &tracing.PlanTrace{ID: p.ID(), TP: p.TP()} - return planTrace -} - func (p *basePhysicalPlan) appendChildCandidate(op *physicalOptimizeOp) { if len(p.Children()) < 1 { return diff --git a/planner/core/plan_cost_ver1.go b/planner/core/plan_cost_ver1.go index 156ac820837f9..2eda230fa8d6f 100644 --- a/planner/core/plan_cost_ver1.go +++ b/planner/core/plan_cost_ver1.go @@ -76,9 +76,9 @@ func (p *PhysicalSelection) getPlanCostVer1(taskType property.TaskType, option * var cpuFactor float64 switch taskType { case property.RootTaskType, property.MppTaskType: - cpuFactor = p.ctx.GetSessionVars().GetCPUFactor() + cpuFactor = p.SCtx().GetSessionVars().GetCPUFactor() case property.CopSingleReadTaskType, property.CopMultiReadTaskType: - cpuFactor = p.ctx.GetSessionVars().GetCopCPUFactor() + cpuFactor = p.SCtx().GetSessionVars().GetCopCPUFactor() default: return 0, errors.Errorf("unknown task type %v", taskType) } @@ -98,7 +98,7 @@ func (p *PhysicalSelection) getPlanCostVer1(taskType property.TaskType, option * // GetCost computes the cost of projection operator itself. func (p *PhysicalProjection) GetCost(count float64) float64 { - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() cpuCost := count * sessVars.GetCPUFactor() concurrency := float64(sessVars.ProjectionConcurrency()) if concurrency <= 0 { @@ -128,7 +128,7 @@ func (p *PhysicalProjection) getPlanCostVer1(taskType property.TaskType, option // GetCost computes cost of index lookup operator itself. func (p *PhysicalIndexLookUpReader) GetCost(costFlag uint64) (cost float64) { indexPlan, tablePlan := p.indexPlan, p.tablePlan - ctx := p.ctx + ctx := p.SCtx() sessVars := ctx.GetSessionVars() // Add cost of building table reader executors. Handles are extracted in batch style, // each handle is a range, the CPU cost of building copTasks should be: @@ -204,21 +204,21 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer1(_ property.TaskType, option // index-side net I/O cost: rows * row-size * net-factor netFactor := getTableNetFactor(p.tablePlan) - rowSize := getTblStats(p.indexPlan).GetAvgRowSize(p.ctx, p.indexPlan.Schema().Columns, true, false) + rowSize := getTblStats(p.indexPlan).GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false) p.planCost += getCardinality(p.indexPlan, costFlag) * rowSize * netFactor // index-side net seek cost p.planCost += estimateNetSeekCost(p.indexPlan) // table-side net I/O cost: rows * row-size * net-factor - tblRowSize := getTblStats(p.tablePlan).GetAvgRowSize(p.ctx, p.tablePlan.Schema().Columns, false, false) + tblRowSize := getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false) p.planCost += getCardinality(p.tablePlan, costFlag) * tblRowSize * netFactor // table-side seek cost p.planCost += estimateNetSeekCost(p.tablePlan) // consider concurrency - p.planCost /= float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) + p.planCost /= float64(p.SCtx().GetSessionVars().DistSQLScanConcurrency()) // lookup-cpu-cost in TiDB p.planCost += p.GetCost(costFlag) @@ -234,7 +234,7 @@ func (p *PhysicalIndexReader) getPlanCostVer1(_ property.TaskType, option *PlanC } var rowCount, rowSize, netFactor, indexPlanCost, netSeekCost float64 - sqlScanConcurrency := p.ctx.GetSessionVars().DistSQLScanConcurrency() + sqlScanConcurrency := p.SCtx().GetSessionVars().DistSQLScanConcurrency() // child's cost childCost, err := p.indexPlan.getPlanCostVer1(property.CopSingleReadTaskType, option) if err != nil { @@ -244,7 +244,7 @@ func (p *PhysicalIndexReader) getPlanCostVer1(_ property.TaskType, option *PlanC p.planCost = indexPlanCost // net I/O cost: rows * row-size * net-factor tblStats := getTblStats(p.indexPlan) - rowSize = tblStats.GetAvgRowSize(p.ctx, p.indexPlan.Schema().Columns, true, false) + rowSize = tblStats.GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false) rowCount = getCardinality(p.indexPlan, costFlag) netFactor = getTableNetFactor(p.indexPlan) p.planCost += rowCount * rowSize * netFactor @@ -264,7 +264,7 @@ func (p *PhysicalIndexReader) getPlanCostVer1(_ property.TaskType, option *PlanC // GetNetDataSize calculates the cost of the plan in network data transfer. func (p *PhysicalIndexReader) GetNetDataSize() float64 { tblStats := getTblStats(p.indexPlan) - rowSize := tblStats.GetAvgRowSize(p.ctx, p.indexPlan.Schema().Columns, true, false) + rowSize := tblStats.GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false) return p.indexPlan.StatsCount() * rowSize } @@ -278,7 +278,7 @@ func (p *PhysicalTableReader) getPlanCostVer1(_ property.TaskType, option *PlanC p.planCost = 0 netFactor := getTableNetFactor(p.tablePlan) var rowCount, rowSize, netSeekCost, tableCost float64 - sqlScanConcurrency := p.ctx.GetSessionVars().DistSQLScanConcurrency() + sqlScanConcurrency := p.SCtx().GetSessionVars().DistSQLScanConcurrency() storeType := p.StoreType switch storeType { case kv.TiKV: @@ -290,7 +290,7 @@ func (p *PhysicalTableReader) getPlanCostVer1(_ property.TaskType, option *PlanC tableCost = childCost p.planCost = childCost // net I/O cost: rows * row-size * net-factor - rowSize = getTblStats(p.tablePlan).GetAvgRowSize(p.ctx, p.tablePlan.Schema().Columns, false, false) + rowSize = getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false) rowCount = getCardinality(p.tablePlan, costFlag) p.planCost += rowCount * rowSize * netFactor // net seek cost @@ -303,7 +303,7 @@ func (p *PhysicalTableReader) getPlanCostVer1(_ property.TaskType, option *PlanC _, isMPP := p.tablePlan.(*PhysicalExchangeSender) if isMPP { // mpp protocol - concurrency = p.ctx.GetSessionVars().CopTiFlashConcurrencyFactor + concurrency = p.SCtx().GetSessionVars().CopTiFlashConcurrencyFactor rowSize = collectRowSizeFromMPPPlan(p.tablePlan) seekCost = accumulateNetSeekCost4MPP(p.tablePlan) childCost, err := p.tablePlan.getPlanCostVer1(property.MppTaskType, option) @@ -313,8 +313,8 @@ func (p *PhysicalTableReader) getPlanCostVer1(_ property.TaskType, option *PlanC p.planCost = childCost } else { // cop protocol - concurrency = float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) - rowSize = getTblStats(p.tablePlan).GetAvgRowSize(p.ctx, p.tablePlan.Schema().Columns, false, false) + concurrency = float64(p.SCtx().GetSessionVars().DistSQLScanConcurrency()) + rowSize = getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false) seekCost = estimateNetSeekCost(p.tablePlan) tType := property.CopSingleReadTaskType childCost, err := p.tablePlan.getPlanCostVer1(tType, option) @@ -331,7 +331,7 @@ func (p *PhysicalTableReader) getPlanCostVer1(_ property.TaskType, option *PlanC // consider concurrency p.planCost /= concurrency // consider tidb_enforce_mpp - if isMPP && p.ctx.GetSessionVars().IsMPPEnforced() && + if isMPP && p.SCtx().GetSessionVars().IsMPPEnforced() && !hasCostFlag(costFlag, CostFlagRecalculate) { // show the real cost in explain-statements p.planCost /= 1000000000 } @@ -347,7 +347,7 @@ func (p *PhysicalTableReader) getPlanCostVer1(_ property.TaskType, option *PlanC // GetNetDataSize calculates the estimated total data size fetched from storage. func (p *PhysicalTableReader) GetNetDataSize() float64 { - rowSize := getTblStats(p.tablePlan).GetAvgRowSize(p.ctx, p.tablePlan.Schema().Columns, false, false) + rowSize := getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false) return p.tablePlan.StatsCount() * rowSize } @@ -367,7 +367,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer1(_ property.TaskType, option * netFactor := getTableNetFactor(tblScan) p.planCost += childCost // child's cost tblStats := getTblStats(tblScan) - rowSize := tblStats.GetAvgRowSize(p.ctx, tblScan.Schema().Columns, false, false) + rowSize := tblStats.GetAvgRowSize(p.SCtx(), tblScan.Schema().Columns, false, false) p.planCost += getCardinality(tblScan, costFlag) * rowSize * netFactor // net I/O cost } for _, partialScan := range p.partialPlans { @@ -386,14 +386,14 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer1(_ property.TaskType, option * netFactor := getTableNetFactor(partialScan) p.planCost += childCost // child's cost tblStats := getTblStats(partialScan) - rowSize := tblStats.GetAvgRowSize(p.ctx, partialScan.Schema().Columns, isIdxScan, false) + rowSize := tblStats.GetAvgRowSize(p.SCtx(), partialScan.Schema().Columns, isIdxScan, false) p.planCost += getCardinality(partialScan, costFlag) * rowSize * netFactor // net I/O cost } // TODO: accumulate table-side seek cost // consider concurrency - copIterWorkers := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) + copIterWorkers := float64(p.SCtx().GetSessionVars().DistSQLScanConcurrency()) p.planCost /= copIterWorkers p.planCostInit = true return p.planCost, nil @@ -402,7 +402,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer1(_ property.TaskType, option * // GetPartialReaderNetDataSize returns the estimated total response data size of a partial read. func (p *PhysicalIndexMergeReader) GetPartialReaderNetDataSize(plan PhysicalPlan) float64 { _, isIdxScan := plan.(*PhysicalIndexScan) - return plan.StatsCount() * getTblStats(plan).GetAvgRowSize(p.ctx, plan.Schema().Columns, isIdxScan, false) + return plan.StatsCount() * getTblStats(plan).GetAvgRowSize(p.SCtx(), plan.Schema().Columns, isIdxScan, false) } // getPlanCostVer1 calculates the cost of the plan if it has not been calculated yet and returns the cost. @@ -414,10 +414,10 @@ func (p *PhysicalTableScan) getPlanCostVer1(_ property.TaskType, option *PlanCos var selfCost float64 var rowCount, rowSize, scanFactor float64 - costModelVersion := p.ctx.GetSessionVars().CostModelVersion - scanFactor = p.ctx.GetSessionVars().GetScanFactor(p.Table) + costModelVersion := p.SCtx().GetSessionVars().CostModelVersion + scanFactor = p.SCtx().GetSessionVars().GetScanFactor(p.Table) if p.Desc && p.prop != nil && p.prop.ExpectedCnt >= smallScanThreshold { - scanFactor = p.ctx.GetSessionVars().GetDescScanFactor(p.Table) + scanFactor = p.SCtx().GetSessionVars().GetDescScanFactor(p.Table) } rowCount = getCardinality(p, costFlag) rowSize = p.getScanRowSize() @@ -439,10 +439,10 @@ func (p *PhysicalIndexScan) getPlanCostVer1(_ property.TaskType, option *PlanCos var selfCost float64 var rowCount, rowSize, scanFactor float64 - costModelVersion := p.ctx.GetSessionVars().CostModelVersion - scanFactor = p.ctx.GetSessionVars().GetScanFactor(p.Table) + costModelVersion := p.SCtx().GetSessionVars().CostModelVersion + scanFactor = p.SCtx().GetSessionVars().GetScanFactor(p.Table) if p.Desc && p.prop != nil && p.prop.ExpectedCnt >= smallScanThreshold { - scanFactor = p.ctx.GetSessionVars().GetDescScanFactor(p.Table) + scanFactor = p.SCtx().GetSessionVars().GetDescScanFactor(p.Table) } rowCount = getCardinality(p, costFlag) rowSize = p.getScanRowSize() @@ -458,7 +458,7 @@ func (p *PhysicalIndexScan) getPlanCostVer1(_ property.TaskType, option *PlanCos // GetCost computes the cost of index join operator and its children. func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost float64, costFlag uint64) float64 { var cpuCost float64 - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() // Add the cost of evaluating outer filter, since inner filter of index join // is always empty, we can simply tell whether outer filter is empty using the // summed length of left/right conditions. @@ -470,7 +470,7 @@ func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost flo innerCPUCost := sessVars.GetCPUFactor() * outerCnt // Cost of sorting and removing duplicate lookup keys: // (outerCnt / batchSize) * (batchSize * Log2(batchSize) + batchSize) * CPUFactor - batchSize := math.Min(float64(p.ctx.GetSessionVars().IndexJoinBatchSize), outerCnt) + batchSize := math.Min(float64(p.SCtx().GetSessionVars().IndexJoinBatchSize), outerCnt) if batchSize > 2 { innerCPUCost += outerCnt * (math.Log2(batchSize) + 1) * sessVars.GetCPUFactor() } @@ -481,7 +481,7 @@ func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost flo // CPU cost of building hash table for inner results: // (outerCnt / batchSize) * (batchSize * distinctFactor) * innerCnt * CPUFactor innerCPUCost += outerCnt * distinctFactor * innerCnt * sessVars.GetCPUFactor() - innerConcurrency := float64(p.ctx.GetSessionVars().IndexLookupJoinConcurrency()) + innerConcurrency := float64(p.SCtx().GetSessionVars().IndexLookupJoinConcurrency()) cpuCost += innerCPUCost / innerConcurrency // Cost of probing hash table in main thread. numPairs := outerCnt * innerCnt @@ -504,7 +504,7 @@ func (p *PhysicalIndexJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost flo memoryCost := innerConcurrency * (batchSize * distinctFactor) * innerCnt * sessVars.GetMemoryFactor() // Cost of inner child plan, i.e, mainly I/O and network cost. innerPlanCost := outerCnt * innerCost - if p.ctx.GetSessionVars().CostModelVersion == 2 { + if p.SCtx().GetSessionVars().CostModelVersion == 2 { // IndexJoin executes a batch of rows at a time, so the actual cost of this part should be // `innerCostPerBatch * numberOfBatches` instead of `innerCostPerRow * numberOfOuterRow`. // Use an empirical value batchRatio to handle this now. @@ -544,7 +544,7 @@ func (p *PhysicalIndexJoin) getPlanCostVer1(taskType property.TaskType, option * // GetCost computes the cost of index merge join operator and its children. func (p *PhysicalIndexHashJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost float64, costFlag uint64) float64 { var cpuCost float64 - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() // Add the cost of evaluating outer filter, since inner filter of index join // is always empty, we can simply tell whether outer filter is empty using the // summed length of left/right conditions. @@ -633,7 +633,7 @@ func (p *PhysicalIndexHashJoin) getPlanCostVer1(taskType property.TaskType, opti // GetCost computes the cost of index merge join operator and its children. func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCost float64, costFlag uint64) float64 { var cpuCost float64 - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() // Add the cost of evaluating outer filter, since inner filter of index join // is always empty, we can simply tell whether outer filter is empty using the // summed length of left/right conditions. @@ -647,7 +647,7 @@ func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCos // (outerCnt / batchSize) * (sortFactor + 1.0) * batchSize * cpuFactor // If `p.NeedOuterSort` is true, the sortFactor is batchSize * Log2(batchSize). // Otherwise, it's 0. - batchSize := math.Min(float64(p.ctx.GetSessionVars().IndexJoinBatchSize), outerCnt) + batchSize := math.Min(float64(p.SCtx().GetSessionVars().IndexJoinBatchSize), outerCnt) sortFactor := 0.0 if p.NeedOuterSort { sortFactor = math.Log2(batchSize) @@ -659,7 +659,7 @@ func (p *PhysicalIndexMergeJoin) GetCost(outerCnt, innerCnt, outerCost, innerCos // (outerCnt / batchSize) * (batchSize * distinctFactor) * cpuFactor // Since we don't know the number of copTasks built, ignore these network cost now. innerCPUCost += outerCnt * distinctFactor * sessVars.GetCPUFactor() - innerConcurrency := float64(p.ctx.GetSessionVars().IndexLookupJoinConcurrency()) + innerConcurrency := float64(p.SCtx().GetSessionVars().IndexLookupJoinConcurrency()) cpuCost += innerCPUCost / innerConcurrency // Cost of merge join in inner worker. numPairs := outerCnt * innerCnt @@ -724,7 +724,7 @@ func (p *PhysicalIndexMergeJoin) getPlanCostVer1(taskType property.TaskType, opt // GetCost computes the cost of apply operator. func (p *PhysicalApply) GetCost(lCount, rCount, lCost, rCost float64) float64 { var cpuCost float64 - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() if len(p.LeftConditions) > 0 { cpuCost += lCount * sessVars.GetCPUFactor() lCount *= SelectionFactor @@ -776,19 +776,19 @@ func (p *PhysicalMergeJoin) GetCost(lCnt, rCnt float64, costFlag uint64) float64 innerCnt := rCnt innerKeys := p.RightJoinKeys innerSchema := p.children[1].Schema() - innerStats := p.children[1].statsInfo() + innerStats := p.children[1].StatsInfo() if p.JoinType == RightOuterJoin { outerCnt = rCnt innerCnt = lCnt innerKeys = p.LeftJoinKeys innerSchema = p.children[0].Schema() - innerStats = p.children[0].statsInfo() + innerStats = p.children[0].StatsInfo() } helper := &fullJoinRowCountHelper{ sctx: p.SCtx(), cartesian: false, - leftProfile: p.children[0].statsInfo(), - rightProfile: p.children[1].statsInfo(), + leftProfile: p.children[0].StatsInfo(), + rightProfile: p.children[1].StatsInfo(), leftJoinKeys: p.LeftJoinKeys, rightJoinKeys: p.RightJoinKeys, leftSchema: p.children[0].Schema(), @@ -806,7 +806,7 @@ func (p *PhysicalMergeJoin) GetCost(lCnt, rCnt float64, costFlag uint64) float64 if hasCostFlag(costFlag, CostFlagUseTrueCardinality) { numPairs = getOperatorActRows(p) } - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() probeCost := numPairs * sessVars.GetCPUFactor() // Cost of evaluating outer filters. var cpuCost float64 @@ -850,10 +850,10 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, _ bool, costFlag uint64, buildCnt, probeCnt = rCnt, lCnt build = p.children[1] } - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() memQuota := sessVars.MemTracker.GetBytesLimit() // sessVars.MemQuotaQuery && hint - rowSize := getAvgRowSize(build.statsInfo(), build.Schema().Columns) + rowSize := getAvgRowSize(build.StatsInfo(), build.Schema().Columns) spill := oomUseTmpStorage && memQuota > 0 && rowSize*buildCnt > float64(memQuota) && p.storeTp != kv.TiFlash // Cost of building hash table. cpuFactor := sessVars.GetCPUFactor() @@ -868,8 +868,8 @@ func (p *PhysicalHashJoin) GetCost(lCnt, rCnt float64, _ bool, costFlag uint64, helper := &fullJoinRowCountHelper{ sctx: p.SCtx(), cartesian: false, - leftProfile: p.children[0].statsInfo(), - rightProfile: p.children[1].statsInfo(), + leftProfile: p.children[0].StatsInfo(), + rightProfile: p.children[1].StatsInfo(), leftJoinKeys: p.LeftJoinKeys, rightJoinKeys: p.RightJoinKeys, leftSchema: p.children[0].Schema(), @@ -961,7 +961,7 @@ func (p *PhysicalHashJoin) getPlanCostVer1(taskType property.TaskType, option *P func (p *PhysicalStreamAgg) GetCost(inputRows float64, isRoot, _ bool, costFlag uint64) float64 { aggFuncFactor := p.getAggFuncCostFactor(false) var cpuCost float64 - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() if isRoot { cpuCost = inputRows * sessVars.GetCPUFactor() * aggFuncFactor } else { @@ -994,7 +994,7 @@ func (p *PhysicalHashAgg) GetCost(inputRows float64, isRoot, isMPP bool, costFla numDistinctFunc := p.numDistinctFunc() aggFuncFactor := p.getAggFuncCostFactor(isMPP) var cpuCost float64 - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() if isRoot { cpuCost = inputRows * sessVars.GetCPUFactor() * aggFuncFactor divisor, con := p.cpuCostDivisor(numDistinctFunc > 0) @@ -1044,13 +1044,13 @@ func (p *PhysicalSort) GetCost(count float64, schema *expression.Schema) float64 if count < 2.0 { count = 2.0 } - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() cpuCost := count * math.Log2(count) * sessVars.GetCPUFactor() memoryCost := count * sessVars.GetMemoryFactor() oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() memQuota := sessVars.MemTracker.GetBytesLimit() // sessVars.MemQuotaQuery && hint - rowSize := getAvgRowSize(p.statsInfo(), schema.Columns) + rowSize := getAvgRowSize(p.StatsInfo(), schema.Columns) spill := oomUseTmpStorage && memQuota > 0 && rowSize*count > float64(memQuota) diskCost := count * sessVars.GetDiskFactor() * rowSize if !spill { @@ -1083,7 +1083,7 @@ func (p *PhysicalTopN) GetCost(count float64, isRoot bool) float64 { if heapSize < 2.0 { heapSize = 2.0 } - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() // Ignore the cost of `doCompaction` in current implementation of `TopNExec`, since it is the // special side-effect of our Chunk format in TiDB layer, which may not exist in coprocessor's // implementation, or may be removed in the future if we change data format. @@ -1122,15 +1122,15 @@ func (p *BatchPointGetPlan) GetCost(opt *physicalOptimizeOp) float64 { if cols == nil { return 0 // the cost of BatchGet generated in fast plan optimization is always 0 } - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() var rowSize, rowCount float64 cost := 0.0 if p.IndexInfo == nil { rowCount = float64(len(p.Handles)) - rowSize = p.stats.HistColl.GetTableAvgRowSize(p.ctx, cols, kv.TiKV, true) + rowSize = p.StatsInfo().HistColl.GetTableAvgRowSize(p.SCtx(), cols, kv.TiKV, true) } else { rowCount = float64(len(p.IndexValues)) - rowSize = p.stats.HistColl.GetIndexAvgRowSize(p.ctx, cols, p.IndexInfo.Unique) + rowSize = p.StatsInfo().HistColl.GetIndexAvgRowSize(p.SCtx(), cols, p.IndexInfo.Unique) } networkFactor := sessVars.GetNetworkFactor(p.TblInfo) seekFactor := sessVars.GetSeekFactor(p.TblInfo) @@ -1162,9 +1162,9 @@ func (p *BatchPointGetPlan) GetAvgRowSize() float64 { return 0 // the cost of BatchGet generated in fast plan optimization is always 0 } if p.IndexInfo == nil { - return p.stats.HistColl.GetTableAvgRowSize(p.ctx, cols, kv.TiKV, true) + return p.StatsInfo().HistColl.GetTableAvgRowSize(p.SCtx(), cols, kv.TiKV, true) } - return p.stats.HistColl.GetIndexAvgRowSize(p.ctx, cols, p.IndexInfo.Unique) + return p.StatsInfo().HistColl.GetIndexAvgRowSize(p.SCtx(), cols, p.IndexInfo.Unique) } // GetCost returns cost of the PointGetPlan. @@ -1173,13 +1173,13 @@ func (p *PointGetPlan) GetCost(opt *physicalOptimizeOp) float64 { if cols == nil { return 0 // the cost of PointGet generated in fast plan optimization is always 0 } - sessVars := p.ctx.GetSessionVars() + sessVars := p.SCtx().GetSessionVars() var rowSize float64 cost := 0.0 if p.IndexInfo == nil { - rowSize = p.stats.HistColl.GetTableAvgRowSize(p.ctx, cols, kv.TiKV, true) + rowSize = p.StatsInfo().HistColl.GetTableAvgRowSize(p.SCtx(), cols, kv.TiKV, true) } else { - rowSize = p.stats.HistColl.GetIndexAvgRowSize(p.ctx, cols, p.IndexInfo.Unique) + rowSize = p.StatsInfo().HistColl.GetIndexAvgRowSize(p.SCtx(), cols, p.IndexInfo.Unique) } networkFactor := sessVars.GetNetworkFactor(p.TblInfo) seekFactor := sessVars.GetSeekFactor(p.TblInfo) @@ -1210,9 +1210,9 @@ func (p *PointGetPlan) GetAvgRowSize() float64 { return 0 // the cost of PointGet generated in fast plan optimization is always 0 } if p.IndexInfo == nil { - return p.stats.HistColl.GetTableAvgRowSize(p.ctx, cols, kv.TiKV, true) + return p.StatsInfo().HistColl.GetTableAvgRowSize(p.SCtx(), cols, kv.TiKV, true) } - return p.stats.HistColl.GetIndexAvgRowSize(p.ctx, cols, p.IndexInfo.Unique) + return p.StatsInfo().HistColl.GetIndexAvgRowSize(p.SCtx(), cols, p.IndexInfo.Unique) } // getPlanCostVer1 calculates the cost of the plan if it has not been calculated yet and returns the cost. @@ -1229,7 +1229,7 @@ func (p *PhysicalUnionAll) getPlanCostVer1(taskType property.TaskType, option *P } childMaxCost = math.Max(childMaxCost, childCost) } - p.planCost = childMaxCost + float64(1+len(p.children))*p.ctx.GetSessionVars().GetConcurrencyFactor() + p.planCost = childMaxCost + float64(1+len(p.children))*p.SCtx().GetSessionVars().GetConcurrencyFactor() p.planCostInit = true return p.planCost, nil } @@ -1246,7 +1246,7 @@ func (p *PhysicalExchangeReceiver) getPlanCostVer1(taskType property.TaskType, o } p.planCost = childCost // accumulate net cost - p.planCost += getCardinality(p.children[0], costFlag) * p.ctx.GetSessionVars().GetNetworkFactor(nil) + p.planCost += getCardinality(p.children[0], costFlag) * p.SCtx().GetSessionVars().GetNetworkFactor(nil) p.planCostInit = true return p.planCost, nil } @@ -1291,11 +1291,11 @@ func estimateNetSeekCost(copTaskPlan PhysicalPlan) float64 { switch x := copTaskPlan.(type) { case *PhysicalTableScan: if x.StoreType == kv.TiFlash { // the old TiFlash interface uses cop-task protocol - return float64(len(x.Ranges)) * float64(len(x.Columns)) * x.ctx.GetSessionVars().GetSeekFactor(x.Table) + return float64(len(x.Ranges)) * float64(len(x.Columns)) * x.SCtx().GetSessionVars().GetSeekFactor(x.Table) } - return float64(len(x.Ranges)) * x.ctx.GetSessionVars().GetSeekFactor(x.Table) // TiKV + return float64(len(x.Ranges)) * x.SCtx().GetSessionVars().GetSeekFactor(x.Table) // TiKV case *PhysicalIndexScan: - return float64(len(x.Ranges)) * x.ctx.GetSessionVars().GetSeekFactor(x.Table) // TiKV + return float64(len(x.Ranges)) * x.SCtx().GetSessionVars().GetSeekFactor(x.Table) // TiKV default: return estimateNetSeekCost(copTaskPlan.Children()[0]) } diff --git a/planner/core/plan_cost_ver2.go b/planner/core/plan_cost_ver2.go index 7985aec6fa00b..c9fd0497a1289 100644 --- a/planner/core/plan_cost_ver2.go +++ b/planner/core/plan_cost_ver2.go @@ -103,7 +103,7 @@ func (p *PhysicalProjection) getPlanCostVer2(taskType property.TaskType, option inputRows := getCardinality(p.children[0], option.CostFlag) cpuFactor := getTaskCPUFactorVer2(p, taskType) - concurrency := float64(p.ctx.GetSessionVars().ProjectionConcurrency()) + concurrency := float64(p.SCtx().GetSessionVars().ProjectionConcurrency()) if concurrency == 0 { concurrency = 1 // un-parallel execution } @@ -129,7 +129,7 @@ func (p *PhysicalIndexScan) getPlanCostVer2(taskType property.TaskType, option * } rows := getCardinality(p, option.CostFlag) - rowSize := math.Max(getAvgRowSize(p.stats, p.schema.Columns), 2.0) // consider all index columns + rowSize := math.Max(getAvgRowSize(p.StatsInfo(), p.schema.Columns), 2.0) // consider all index columns scanFactor := getTaskScanFactorVer2(p, kv.TiKV, taskType) p.planCostVer2 = scanCostVer2(option, rows, rowSize, scanFactor) @@ -148,9 +148,9 @@ func (p *PhysicalTableScan) getPlanCostVer2(taskType property.TaskType, option * rows := getCardinality(p, option.CostFlag) var rowSize float64 if p.StoreType == kv.TiKV { - rowSize = getAvgRowSize(p.stats, p.tblCols) // consider all columns if TiKV + rowSize = getAvgRowSize(p.StatsInfo(), p.tblCols) // consider all columns if TiKV } else { // TiFlash - rowSize = getAvgRowSize(p.stats, p.schema.Columns) + rowSize = getAvgRowSize(p.StatsInfo(), p.schema.Columns) } rowSize = math.Max(rowSize, 2.0) scanFactor := getTaskScanFactorVer2(p, p.StoreType, taskType) @@ -175,9 +175,9 @@ func (p *PhysicalIndexReader) getPlanCostVer2(taskType property.TaskType, option } rows := getCardinality(p.indexPlan, option.CostFlag) - rowSize := getAvgRowSize(p.stats, p.schema.Columns) + rowSize := getAvgRowSize(p.StatsInfo(), p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) - concurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) + concurrency := float64(p.SCtx().GetSessionVars().DistSQLScanConcurrency()) netCost := netCostVer2(option, rows, rowSize, netFactor) @@ -200,9 +200,9 @@ func (p *PhysicalTableReader) getPlanCostVer2(taskType property.TaskType, option } rows := getCardinality(p.tablePlan, option.CostFlag) - rowSize := getAvgRowSize(p.stats, p.schema.Columns) + rowSize := getAvgRowSize(p.StatsInfo(), p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) - concurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) + concurrency := float64(p.SCtx().GetSessionVars().DistSQLScanConcurrency()) childType := property.CopSingleReadTaskType if p.StoreType == kv.TiFlash { // mpp protocol childType = property.MppTaskType @@ -219,7 +219,7 @@ func (p *PhysicalTableReader) getPlanCostVer2(taskType property.TaskType, option p.planCostInit = true // consider tidb_enforce_mpp - if p.StoreType == kv.TiFlash && p.ctx.GetSessionVars().IsMPPEnforced() && + if p.StoreType == kv.TiFlash && p.SCtx().GetSessionVars().IsMPPEnforced() && !hasCostFlag(option.CostFlag, CostFlagRecalculate) { // show the real cost in explain-statements p.planCostVer2 = divCostVer2(p.planCostVer2, 1000000000) } @@ -241,13 +241,13 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType, indexRows := getCardinality(p.indexPlan, option.CostFlag) tableRows := getCardinality(p.indexPlan, option.CostFlag) - indexRowSize := getTblStats(p.indexPlan).GetAvgRowSize(p.ctx, p.indexPlan.Schema().Columns, true, false) - tableRowSize := getTblStats(p.tablePlan).GetAvgRowSize(p.ctx, p.tablePlan.Schema().Columns, false, false) + indexRowSize := getTblStats(p.indexPlan).GetAvgRowSize(p.SCtx(), p.indexPlan.Schema().Columns, true, false) + tableRowSize := getTblStats(p.tablePlan).GetAvgRowSize(p.SCtx(), p.tablePlan.Schema().Columns, false, false) cpuFactor := getTaskCPUFactorVer2(p, taskType) netFactor := getTaskNetFactorVer2(p, taskType) requestFactor := getTaskRequestFactorVer2(p, taskType) - distConcurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) - doubleReadConcurrency := float64(p.ctx.GetSessionVars().IndexLookupConcurrency()) + distConcurrency := float64(p.SCtx().GetSessionVars().DistSQLScanConcurrency()) + doubleReadConcurrency := float64(p.SCtx().GetSessionVars().IndexLookupConcurrency()) // index-side indexNetCost := netCostVer2(option, indexRows, indexRowSize, netFactor) @@ -269,7 +269,7 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType, doubleReadCPUCost := newCostVer2(option, cpuFactor, indexRows*cpuFactor.Value, func() string { return fmt.Sprintf("double-read-cpu(%v*%v)", doubleReadRows, cpuFactor) }) - batchSize := float64(p.ctx.GetSessionVars().IndexLookupSize) + batchSize := float64(p.SCtx().GetSessionVars().IndexLookupSize) taskPerBatch := 32.0 // TODO: remove this magic number doubleReadTasks := doubleReadRows / batchSize * taskPerBatch doubleReadRequestCost := doubleReadCostVer2(option, doubleReadTasks, requestFactor) @@ -277,7 +277,7 @@ func (p *PhysicalIndexLookUpReader) getPlanCostVer2(taskType property.TaskType, p.planCostVer2 = sumCostVer2(indexSideCost, divCostVer2(sumCostVer2(tableSideCost, doubleReadCost), doubleReadConcurrency)) - if p.ctx.GetSessionVars().EnablePaging && p.expectedCnt > 0 && p.expectedCnt <= paging.Threshold { + if p.SCtx().GetSessionVars().EnablePaging && p.expectedCnt > 0 && p.expectedCnt <= paging.Threshold { // if the expectCnt is below the paging threshold, using paging API p.Paging = true // TODO: move this operation from cost model to physical optimization p.planCostVer2 = mulCostVer2(p.planCostVer2, 0.6) @@ -297,12 +297,12 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer2(taskType property.TaskType, o } netFactor := getTaskNetFactorVer2(p, taskType) - distConcurrency := float64(p.ctx.GetSessionVars().DistSQLScanConcurrency()) + distConcurrency := float64(p.SCtx().GetSessionVars().DistSQLScanConcurrency()) var tableSideCost costVer2 if tablePath := p.tablePlan; tablePath != nil { rows := getCardinality(tablePath, option.CostFlag) - rowSize := getAvgRowSize(tablePath.Stats(), tablePath.Schema().Columns) + rowSize := getAvgRowSize(tablePath.StatsInfo(), tablePath.Schema().Columns) tableNetCost := netCostVer2(option, rows, rowSize, netFactor) tableChildCost, err := tablePath.getPlanCostVer2(taskType, option) @@ -315,7 +315,7 @@ func (p *PhysicalIndexMergeReader) getPlanCostVer2(taskType property.TaskType, o indexSideCost := make([]costVer2, 0, len(p.partialPlans)) for _, indexPath := range p.partialPlans { rows := getCardinality(indexPath, option.CostFlag) - rowSize := getAvgRowSize(indexPath.Stats(), indexPath.Schema().Columns) + rowSize := getAvgRowSize(indexPath.StatsInfo(), indexPath.Schema().Columns) indexNetCost := netCostVer2(option, rows, rowSize, netFactor) indexChildCost, err := indexPath.getPlanCostVer2(taskType, option) @@ -347,12 +347,12 @@ func (p *PhysicalSort) getPlanCostVer2(taskType property.TaskType, option *PlanC } rows := math.Max(getCardinality(p.children[0], option.CostFlag), 1) - rowSize := getAvgRowSize(p.statsInfo(), p.Schema().Columns) + rowSize := getAvgRowSize(p.StatsInfo(), p.Schema().Columns) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) diskFactor := defaultVer2Factors.TiDBDisk oomUseTmpStorage := variable.EnableTmpStorageOnOOM.Load() - memQuota := p.ctx.GetSessionVars().MemTracker.GetBytesLimit() + memQuota := p.SCtx().GetSessionVars().MemTracker.GetBytesLimit() spill := taskType == property.RootTaskType && // only TiDB can spill oomUseTmpStorage && // spill is enabled memQuota > 0 && // mem-quota is set @@ -396,7 +396,7 @@ func (p *PhysicalTopN) getPlanCostVer2(taskType property.TaskType, option *PlanC rows := getCardinality(p.children[0], option.CostFlag) n := math.Max(1, float64(p.Count+p.Offset)) - rowSize := getAvgRowSize(p.statsInfo(), p.Schema().Columns) + rowSize := getAvgRowSize(p.StatsInfo(), p.Schema().Columns) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) @@ -447,10 +447,10 @@ func (p *PhysicalHashAgg) getPlanCostVer2(taskType property.TaskType, option *Pl inputRows := getCardinality(p.children[0], option.CostFlag) outputRows := getCardinality(p, option.CostFlag) - outputRowSize := getAvgRowSize(p.Stats(), p.Schema().Columns) + outputRowSize := getAvgRowSize(p.StatsInfo(), p.Schema().Columns) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) - concurrency := float64(p.ctx.GetSessionVars().HashAggFinalConcurrency()) + concurrency := float64(p.SCtx().GetSessionVars().HashAggFinalConcurrency()) aggCost := aggCostVer2(option, inputRows, p.AggFuncs, cpuFactor) groupCost := groupCostVer2(option, inputRows, p.GroupByItems, cpuFactor) @@ -518,7 +518,7 @@ func (p *PhysicalHashJoin) getPlanCostVer2(taskType property.TaskType, option *P } buildRows := getCardinality(build, option.CostFlag) probeRows := getCardinality(probe, option.CostFlag) - buildRowSize := getAvgRowSize(build.Stats(), build.Schema().Columns) + buildRowSize := getAvgRowSize(build.StatsInfo(), build.Schema().Columns) tidbConcurrency := float64(p.Concurrency) mppConcurrency := float64(3) // TODO: remove this empirical value cpuFactor := getTaskCPUFactorVer2(p, taskType) @@ -560,12 +560,12 @@ func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, opt build, probe := p.children[1-p.InnerChildIdx], p.children[p.InnerChildIdx] buildRows := getCardinality(build, option.CostFlag) - buildRowSize := getAvgRowSize(build.Stats(), build.Schema().Columns) + buildRowSize := getAvgRowSize(build.StatsInfo(), build.Schema().Columns) probeRowsOne := getCardinality(probe, option.CostFlag) probeRowsTot := probeRowsOne * buildRows - probeRowSize := getAvgRowSize(probe.Stats(), probe.Schema().Columns) + probeRowSize := getAvgRowSize(probe.StatsInfo(), probe.Schema().Columns) buildFilters, probeFilters := p.LeftConditions, p.RightConditions - probeConcurrency := float64(p.ctx.GetSessionVars().IndexLookupJoinConcurrency()) + probeConcurrency := float64(p.SCtx().GetSessionVars().IndexLookupJoinConcurrency()) cpuFactor := getTaskCPUFactorVer2(p, taskType) memFactor := getTaskMemFactorVer2(p, taskType) requestFactor := getTaskRequestFactorVer2(p, taskType) @@ -607,12 +607,12 @@ func (p *PhysicalIndexJoin) getIndexJoinCostVer2(taskType property.TaskType, opt // Double Read Cost doubleReadCost := newZeroCostVer2(traceCost(option)) - if p.ctx.GetSessionVars().IndexJoinDoubleReadPenaltyCostRate > 0 { - batchSize := float64(p.ctx.GetSessionVars().IndexJoinBatchSize) + if p.SCtx().GetSessionVars().IndexJoinDoubleReadPenaltyCostRate > 0 { + batchSize := float64(p.SCtx().GetSessionVars().IndexJoinBatchSize) taskPerBatch := 1024.0 // TODO: remove this magic number doubleReadTasks := buildRows / batchSize * taskPerBatch doubleReadCost = doubleReadCostVer2(option, doubleReadTasks, requestFactor) - doubleReadCost = mulCostVer2(doubleReadCost, p.ctx.GetSessionVars().IndexJoinDoubleReadPenaltyCostRate) + doubleReadCost = mulCostVer2(doubleReadCost, p.SCtx().GetSessionVars().IndexJoinDoubleReadPenaltyCostRate) } p.planCostVer2 = sumCostVer2(startCost, buildChildCost, buildFilterCost, buildTaskCost, divCostVer2(sumCostVer2(doubleReadCost, probeCost, probeFilterCost, hashTableCost), probeConcurrency)) @@ -674,7 +674,7 @@ func (p *PhysicalUnionAll) getPlanCostVer2(taskType property.TaskType, option *P return p.planCostVer2, nil } - concurrency := float64(p.ctx.GetSessionVars().UnionConcurrency()) + concurrency := float64(p.SCtx().GetSessionVars().UnionConcurrency()) childCosts := make([]costVer2, 0, len(p.children)) for _, child := range p.children { childCost, err := child.getPlanCostVer2(taskType, option) @@ -696,7 +696,7 @@ func (p *PhysicalExchangeReceiver) getPlanCostVer2(taskType property.TaskType, o } rows := getCardinality(p, option.CostFlag) - rowSize := getAvgRowSize(p.stats, p.Schema().Columns) + rowSize := getAvgRowSize(p.StatsInfo(), p.Schema().Columns) netFactor := getTaskNetFactorVer2(p, taskType) isBCast := false if sender, ok := p.children[0].(*PhysicalExchangeSender); ok { @@ -729,7 +729,7 @@ func (p *PointGetPlan) getPlanCostVer2(taskType property.TaskType, option *PlanC p.planCostInit = true return zeroCostVer2, nil } - rowSize := getAvgRowSize(p.stats, p.schema.Columns) + rowSize := getAvgRowSize(p.StatsInfo(), p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) p.planCostVer2 = netCostVer2(option, 1, rowSize, netFactor) @@ -749,7 +749,7 @@ func (p *BatchPointGetPlan) getPlanCostVer2(taskType property.TaskType, option * return zeroCostVer2, nil } rows := getCardinality(p, option.CostFlag) - rowSize := getAvgRowSize(p.stats, p.schema.Columns) + rowSize := getAvgRowSize(p.StatsInfo(), p.schema.Columns) netFactor := getTaskNetFactorVer2(p, taskType) p.planCostVer2 = netCostVer2(option, rows, rowSize, netFactor) diff --git a/planner/core/plan_stats_test.go b/planner/core/plan_stats_test.go index ef60e63417203..2de22416a333d 100644 --- a/planner/core/plan_stats_test.go +++ b/planner/core/plan_stats_test.go @@ -70,7 +70,7 @@ func TestPlanStatsLoad(t *testing.T) { check: func(p plannercore.Plan, tableInfo *model.TableInfo) { switch pp := p.(type) { case *plannercore.PhysicalTableReader: - stats := pp.Stats().HistColl + stats := pp.StatsInfo().HistColl require.Equal(t, 0, countFullStats(stats, tableInfo.Columns[1].ID)) require.Greater(t, countFullStats(stats, tableInfo.Columns[2].ID), 0) default: @@ -84,7 +84,7 @@ func TestPlanStatsLoad(t *testing.T) { pua, ok := p.(*plannercore.PhysicalUnionAll) require.True(t, ok) for _, child := range pua.Children() { - require.Greater(t, countFullStats(child.Stats().HistColl, tableInfo.Columns[2].ID), 0) + require.Greater(t, countFullStats(child.StatsInfo().HistColl, tableInfo.Columns[2].ID), 0) } }, }, @@ -93,8 +93,8 @@ func TestPlanStatsLoad(t *testing.T) { check: func(p plannercore.Plan, tableInfo *model.TableInfo) { pp, ok := p.(plannercore.PhysicalPlan) require.True(t, ok) - require.Greater(t, countFullStats(pp.Children()[0].Stats().HistColl, tableInfo.Columns[3].ID), 0) - require.Greater(t, countFullStats(pp.Children()[1].Stats().HistColl, tableInfo.Columns[3].ID), 0) + require.Greater(t, countFullStats(pp.Children()[0].StatsInfo().HistColl, tableInfo.Columns[3].ID), 0) + require.Greater(t, countFullStats(pp.Children()[1].StatsInfo().HistColl, tableInfo.Columns[3].ID), 0) }, }, { // Apply @@ -106,8 +106,8 @@ func TestPlanStatsLoad(t *testing.T) { require.True(t, ok) left := pa.PhysicalHashJoin.Children()[0] right := pa.PhysicalHashJoin.Children()[0] - require.Greater(t, countFullStats(left.Stats().HistColl, tableInfo.Columns[2].ID), 0) - require.Greater(t, countFullStats(right.Stats().HistColl, tableInfo.Columns[3].ID), 0) + require.Greater(t, countFullStats(left.StatsInfo().HistColl, tableInfo.Columns[2].ID), 0) + require.Greater(t, countFullStats(right.StatsInfo().HistColl, tableInfo.Columns[3].ID), 0) }, }, { // > Any @@ -117,7 +117,7 @@ func TestPlanStatsLoad(t *testing.T) { require.True(t, ok) ptr, ok := ph.Children()[0].(*plannercore.PhysicalTableReader) require.True(t, ok) - require.Greater(t, countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), 0) + require.Greater(t, countFullStats(ptr.StatsInfo().HistColl, tableInfo.Columns[2].ID), 0) }, }, { // in @@ -127,7 +127,7 @@ func TestPlanStatsLoad(t *testing.T) { require.True(t, ok) ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) require.True(t, ok) - require.Greater(t, countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), 0) + require.Greater(t, countFullStats(ptr.StatsInfo().HistColl, tableInfo.Columns[2].ID), 0) }, }, { // not in @@ -137,7 +137,7 @@ func TestPlanStatsLoad(t *testing.T) { require.True(t, ok) ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) require.True(t, ok) - require.Greater(t, countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), 0) + require.Greater(t, countFullStats(ptr.StatsInfo().HistColl, tableInfo.Columns[2].ID), 0) }, }, { // exists @@ -147,7 +147,7 @@ func TestPlanStatsLoad(t *testing.T) { require.True(t, ok) ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) require.True(t, ok) - require.Greater(t, countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), 0) + require.Greater(t, countFullStats(ptr.StatsInfo().HistColl, tableInfo.Columns[2].ID), 0) }, }, { // not exists @@ -157,7 +157,7 @@ func TestPlanStatsLoad(t *testing.T) { require.True(t, ok) ptr, ok := ph.Children()[1].(*plannercore.PhysicalTableReader) require.True(t, ok) - require.Greater(t, countFullStats(ptr.Stats().HistColl, tableInfo.Columns[2].ID), 0) + require.Greater(t, countFullStats(ptr.StatsInfo().HistColl, tableInfo.Columns[2].ID), 0) }, }, { // CTE @@ -171,7 +171,7 @@ func TestPlanStatsLoad(t *testing.T) { require.True(t, ok) reader, ok := pp.Children()[0].(*plannercore.PhysicalTableReader) require.True(t, ok) - require.Greater(t, countFullStats(reader.Stats().HistColl, tableInfo.Columns[2].ID), 0) + require.Greater(t, countFullStats(reader.StatsInfo().HistColl, tableInfo.Columns[2].ID), 0) }, }, { // recursive CTE @@ -183,7 +183,7 @@ func TestPlanStatsLoad(t *testing.T) { require.True(t, ok) reader, ok := pp.Children()[0].(*plannercore.PhysicalTableReader) require.True(t, ok) - require.Greater(t, countFullStats(reader.Stats().HistColl, tableInfo.Columns[2].ID), 0) + require.Greater(t, countFullStats(reader.StatsInfo().HistColl, tableInfo.Columns[2].ID), 0) }, }, { // check idx(b) @@ -193,7 +193,7 @@ func TestPlanStatsLoad(t *testing.T) { require.True(t, ok) pis, ok := pr.IndexPlans[0].(*plannercore.PhysicalIndexScan) require.True(t, ok) - require.True(t, pis.Stats().HistColl.Indices[1].IsEssentialStatsLoaded()) + require.True(t, pis.StatsInfo().HistColl.Indices[1].IsEssentialStatsLoaded()) }, }, } @@ -287,7 +287,7 @@ func TestPlanStatsLoadTimeout(t *testing.T) { require.NoError(t, err) // not fail sql for timeout when pseudo=true switch pp := plan.(type) { case *plannercore.PhysicalTableReader: - stats := pp.Stats().HistColl + stats := pp.StatsInfo().HistColl require.Equal(t, 0, countFullStats(stats, tableInfo.Columns[0].ID)) require.Equal(t, 0, countFullStats(stats, tableInfo.Columns[2].ID)) // pseudo stats default: diff --git a/planner/core/plan_to_pb.go b/planner/core/plan_to_pb.go index 6e2f545b80645..a7b885674b0ca 100644 --- a/planner/core/plan_to_pb.go +++ b/planner/core/plan_to_pb.go @@ -30,7 +30,7 @@ import ( // ToPB implements PhysicalPlan ToPB interface. func (p *basePhysicalPlan) ToPB(_ sessionctx.Context, _ kv.StoreType) (*tipb.Executor, error) { - return nil, errors.Errorf("plan %s fails converts to PB", p.basePlan.ExplainID()) + return nil, errors.Errorf("plan %s fails converts to PB", p.Plan.ExplainID()) } // ToPB implements PhysicalPlan ToPB interface. @@ -247,7 +247,7 @@ func (p *PhysicalLimit) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*t // ToPB implements PhysicalPlan ToPB interface. func (p *PhysicalTableScan) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { - if storeType == kv.TiFlash && p.Table.GetPartitionInfo() != nil && p.IsMPPOrBatchCop && p.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { + if storeType == kv.TiFlash && p.Table.GetPartitionInfo() != nil && p.IsMPPOrBatchCop && p.SCtx().GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { return p.partitionTableScanToPBForFlash(ctx) } tsExec := tables.BuildTableScanFromInfos(p.Table, p.Columns) @@ -727,7 +727,7 @@ func (p *PhysicalWindow) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (* // ToPB implements PhysicalPlan ToPB interface. func (p *PhysicalSort) ToPB(ctx sessionctx.Context, storeType kv.StoreType) (*tipb.Executor, error) { if !p.IsPartialSort { - return nil, errors.Errorf("sort %s can't convert to pb, because it isn't a partial sort", p.basePlan.ExplainID()) + return nil, errors.Errorf("sort %s can't convert to pb, because it isn't a partial sort", p.Plan.ExplainID()) } sc := ctx.GetSessionVars().StmtCtx diff --git a/planner/core/planbuilder.go b/planner/core/planbuilder.go index 05db11ba202f7..c9e16214dee78 100644 --- a/planner/core/planbuilder.go +++ b/planner/core/planbuilder.go @@ -1789,7 +1789,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(_ context.Context, dbName m tblColHists: &(statistics.PseudoTable(tblInfo)).HistColl, }.Init(b.ctx, b.getSelectOffset()) // There is no alternative plan choices, so just use pseudo stats to avoid panic. - is.stats = &property.StatsInfo{HistColl: &(statistics.PseudoTable(tblInfo)).HistColl} + is.SetStats(&property.StatsInfo{HistColl: &(statistics.PseudoTable(tblInfo)).HistColl}) if hasCommonCols { for _, c := range commonInfos { is.Columns = append(is.Columns, c.ColumnInfo) @@ -1840,7 +1840,7 @@ func (b *PlanBuilder) buildPhysicalIndexLookUpReader(_ context.Context, dbName m cop := &copTask{ indexPlan: is, tablePlan: ts, - tblColHists: is.stats.HistColl, + tblColHists: is.StatsInfo().HistColl, extraHandleCol: extraCol, commonHandleCols: commonCols, } @@ -4284,7 +4284,7 @@ func (b *PlanBuilder) buildSelectPlanOfInsert(ctx context.Context, insert *ast.I } for i := range schema4NewRow.Columns { if schema4NewRow.Columns[i] == nil { - schema4NewRow.Columns[i] = &expression.Column{UniqueID: insertPlan.ctx.GetSessionVars().AllocPlanColumnID()} + schema4NewRow.Columns[i] = &expression.Column{UniqueID: insertPlan.SCtx().GetSessionVars().AllocPlanColumnID()} names4NewRow[i] = types.EmptyName } } @@ -5114,7 +5114,7 @@ func (b *PlanBuilder) buildExplainPlan(targetPlan Plan, format string, explainRo ExplainRows: explainRows, RuntimeStatsColl: runtimeStats, } - p.ctx = b.ctx + p.SetSCtx(b.ctx) return p, p.prepareSchema() } diff --git a/planner/core/point_get_plan.go b/planner/core/point_get_plan.go index 4ca4d44057a8e..bf6190575858f 100644 --- a/planner/core/point_get_plan.go +++ b/planner/core/point_get_plan.go @@ -33,6 +33,7 @@ import ( "github.com/pingcap/tidb/parser/opcode" "github.com/pingcap/tidb/parser/terror" ptypes "github.com/pingcap/tidb/parser/types" + "github.com/pingcap/tidb/planner/core/internal/base" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/privilege" "github.com/pingcap/tidb/sessionctx" @@ -61,7 +62,7 @@ import ( // When we detect that the statement has a unique equal access condition, this plan is used. // This plan is much faster to build and to execute because it avoid the optimization and coprocessor cost. type PointGetPlan struct { - basePlan + base.Plan dbName string schema *expression.Schema TblInfo *model.TableInfo @@ -102,7 +103,7 @@ func (p *PointGetPlan) getEstRowCountForDisplay() float64 { if p == nil { return 0 } - return p.statsInfo().RowCount * getEstimatedProbeCntFromProbeParents(p.probeParents) + return p.StatsInfo().RowCount * getEstimatedProbeCntFromProbeParents(p.probeParents) } func (p *PointGetPlan) getActualProbeCnt(statsColl *execdetails.RuntimeStatsColl) int64 { @@ -215,13 +216,13 @@ func (*PointGetPlan) StatsCount() float64 { return 1 } -// statsInfo will return the the RowCount of property.StatsInfo for this plan. -func (p *PointGetPlan) statsInfo() *property.StatsInfo { - if p.stats == nil { - p.stats = &property.StatsInfo{} +// StatsInfo will return the the RowCount of property.StatsInfo for this plan. +func (p *PointGetPlan) StatsInfo() *property.StatsInfo { + if p.Plan.StatsInfo() == nil { + p.Plan.SetStats(&property.StatsInfo{}) } - p.stats.RowCount = 1 - return p.stats + p.Plan.StatsInfo().RowCount = 1 + return p.Plan.StatsInfo() } // Children gets all the children. @@ -260,7 +261,7 @@ func (p *PointGetPlan) MemoryUsage() (sum int64) { return } - sum = emptyPointGetPlanSize + p.basePlan.MemoryUsage() + int64(len(p.dbName)) + int64(cap(p.IdxColLens))*size.SizeOfInt + + sum = emptyPointGetPlanSize + p.Plan.MemoryUsage() + int64(len(p.dbName)) + int64(cap(p.IdxColLens))*size.SizeOfInt + int64(cap(p.IndexConstants)+cap(p.ColsFieldType)+cap(p.IdxCols)+cap(p.outputNames)+cap(p.Columns)+cap(p.accessCols))*size.SizeOfPointer if p.schema != nil { sum += p.schema.MemoryUsage() @@ -352,7 +353,7 @@ func (p *BatchPointGetPlan) getEstRowCountForDisplay() float64 { if p == nil { return 0 } - return p.statsInfo().RowCount * getEstimatedProbeCntFromProbeParents(p.probeParents) + return p.StatsInfo().RowCount * getEstimatedProbeCntFromProbeParents(p.probeParents) } func (p *BatchPointGetPlan) getActualProbeCnt(statsColl *execdetails.RuntimeStatsColl) int64 { @@ -440,12 +441,12 @@ func (*BatchPointGetPlan) GetChildReqProps(_ int) *property.PhysicalProperty { // StatsCount will return the the RowCount of property.StatsInfo for this plan. func (p *BatchPointGetPlan) StatsCount() float64 { - return p.statsInfo().RowCount + return p.Plan.StatsInfo().RowCount } -// statsInfo will return the the RowCount of property.StatsInfo for this plan. -func (p *BatchPointGetPlan) statsInfo() *property.StatsInfo { - return p.stats +// StatsInfo will return the the RowCount of property.StatsInfo for this plan. +func (p *BatchPointGetPlan) StatsInfo() *property.StatsInfo { + return p.Plan.StatsInfo() } // Children gets all the children. @@ -549,7 +550,7 @@ func TryFastPlan(ctx sessionctx.Context, node ast.Node) (p Plan) { if vars.StmtCtx.OptimizeTracer == nil { vars.StmtCtx.OptimizeTracer = &tracing.OptimizeTracer{} } - vars.StmtCtx.OptimizeTracer.SetFastPlan(p.buildPlanTrace()) + vars.StmtCtx.OptimizeTracer.SetFastPlan(p.BuildPlanTrace()) } }() // Try to convert the `SELECT a, b, c FROM t WHERE (a, b, c) in ((1, 2, 4), (1, 3, 5))` to @@ -1203,7 +1204,7 @@ func partitionNameInSet(name model.CIStr, pnames []model.CIStr) bool { func newPointGetPlan(ctx sessionctx.Context, dbName string, schema *expression.Schema, tbl *model.TableInfo, names []*types.FieldName) *PointGetPlan { p := &PointGetPlan{ - basePlan: newBasePlan(ctx, plancodec.TypePointGet, 0), + Plan: base.NewBasePlan(ctx, plancodec.TypePointGet, 0), dbName: dbName, schema: schema, TblInfo: tbl, diff --git a/planner/core/rule_aggregation_elimination.go b/planner/core/rule_aggregation_elimination.go index 44ca858f3bdf7..cf4494c5d6f32 100644 --- a/planner/core/rule_aggregation_elimination.go +++ b/planner/core/rule_aggregation_elimination.go @@ -183,9 +183,9 @@ func CheckCanConvertAggToProj(agg *LogicalAggregation) bool { func ConvertAggToProj(agg *LogicalAggregation, schema *expression.Schema) (bool, *LogicalProjection) { proj := LogicalProjection{ Exprs: make([]expression.Expression, 0, len(agg.AggFuncs)), - }.Init(agg.ctx, agg.blockOffset) + }.Init(agg.SCtx(), agg.SelectBlockOffset()) for _, fun := range agg.AggFuncs { - ok, expr := rewriteExpr(agg.ctx, fun) + ok, expr := rewriteExpr(agg.SCtx(), fun) if !ok { return false, nil } diff --git a/planner/core/rule_aggregation_push_down.go b/planner/core/rule_aggregation_push_down.go index 1cff0100f3b40..d262787c51235 100644 --- a/planner/core/rule_aggregation_push_down.go +++ b/planner/core/rule_aggregation_push_down.go @@ -143,7 +143,7 @@ func (a *aggregationPushDownSolver) collectAggFuncs(agg *LogicalAggregation, joi // treated as group by columns in join subquery. func (a *aggregationPushDownSolver) collectGbyCols(agg *LogicalAggregation, join *LogicalJoin) (leftGbyCols, rightGbyCols []*expression.Column) { leftChild := join.children[0] - ctx := agg.ctx + ctx := agg.SCtx() for _, gbyExpr := range agg.GroupByItems { cols := expression.ExtractColumns(gbyExpr) for _, col := range cols { @@ -265,7 +265,7 @@ func (a *aggregationPushDownSolver) tryToPushDownAgg(oldAgg *LogicalAggregation, } nullGenerating := (join.JoinType == LeftOuterJoin && childIdx == 1) || (join.JoinType == RightOuterJoin && childIdx == 0) - agg, err := a.makeNewAgg(join.ctx, aggFuncs, gbyCols, aggHints, blockOffset, nullGenerating) + agg, err := a.makeNewAgg(join.SCtx(), aggFuncs, gbyCols, aggHints, blockOffset, nullGenerating) if err != nil { return nil, err } @@ -291,7 +291,7 @@ func (a *aggregationPushDownSolver) tryToPushDownAgg(oldAgg *LogicalAggregation, func (*aggregationPushDownSolver) getDefaultValues(agg *LogicalAggregation) ([]types.Datum, bool) { defaultValues := make([]types.Datum, 0, agg.Schema().Len()) for _, aggFunc := range agg.AggFuncs { - value, existsDefaultValue := aggFunc.EvalNullValueInOuterJoin(agg.ctx, agg.children[0].Schema()) + value, existsDefaultValue := aggFunc.EvalNullValueInOuterJoin(agg.SCtx(), agg.children[0].Schema()) if !existsDefaultValue { return nil, false } @@ -339,7 +339,7 @@ func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs newAggFuncDescs = append(newAggFuncDescs, newFuncs...) } for _, gbyCol := range gbyCols { - firstRow, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncFirstRow, []expression.Expression{gbyCol}, false) + firstRow, err := aggregation.NewAggFuncDesc(agg.SCtx(), ast.AggFuncFirstRow, []expression.Expression{gbyCol}, false) if err != nil { return nil, err } @@ -356,7 +356,7 @@ func (a *aggregationPushDownSolver) makeNewAgg(ctx sessionctx.Context, aggFuncs } func (*aggregationPushDownSolver) splitPartialAgg(agg *LogicalAggregation) (pushedAgg *LogicalAggregation) { - partial, final, _ := BuildFinalModeAggregation(agg.ctx, &AggInfo{ + partial, final, _ := BuildFinalModeAggregation(agg.SCtx(), &AggInfo{ AggFuncs: agg.AggFuncs, GroupByItems: agg.GroupByItems, Schema: agg.schema, @@ -373,7 +373,7 @@ func (*aggregationPushDownSolver) splitPartialAgg(agg *LogicalAggregation) (push AggFuncs: partial.AggFuncs, GroupByItems: partial.GroupByItems, aggHints: agg.aggHints, - }.Init(agg.ctx, agg.blockOffset) + }.Init(agg.SCtx(), agg.SelectBlockOffset()) pushedAgg.SetSchema(partial.Schema) return } @@ -381,12 +381,12 @@ func (*aggregationPushDownSolver) splitPartialAgg(agg *LogicalAggregation) (push // pushAggCrossUnion will try to push the agg down to the union. If the new aggregation's group-by columns doesn't contain unique key. // We will return the new aggregation. Otherwise we will transform the aggregation to projection. func (*aggregationPushDownSolver) pushAggCrossUnion(agg *LogicalAggregation, unionSchema *expression.Schema, unionChild LogicalPlan) (LogicalPlan, error) { - ctx := agg.ctx + ctx := agg.SCtx() newAgg := LogicalAggregation{ AggFuncs: make([]*aggregation.AggFuncDesc, 0, len(agg.AggFuncs)), GroupByItems: make([]expression.Expression, 0, len(agg.GroupByItems)), aggHints: agg.aggHints, - }.Init(ctx, agg.blockOffset) + }.Init(ctx, agg.SelectBlockOffset()) newAgg.SetSchema(agg.schema.Clone()) for _, aggFunc := range agg.AggFuncs { newAggFunc := aggFunc.Clone() @@ -401,7 +401,7 @@ func (*aggregationPushDownSolver) pushAggCrossUnion(agg *LogicalAggregation, uni newExpr := expression.ColumnSubstitute(gbyExpr, unionSchema, expression.Column2Exprs(unionChild.Schema().Columns)) newAgg.GroupByItems = append(newAgg.GroupByItems, newExpr) // TODO: if there is a duplicated first_row function, we can delete it. - firstRow, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncFirstRow, []expression.Expression{gbyExpr}, false) + firstRow, err := aggregation.NewAggFuncDesc(agg.SCtx(), ast.AggFuncFirstRow, []expression.Expression{gbyExpr}, false) if err != nil { return nil, err } @@ -489,7 +489,7 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan, opt *logicalOptim if rightInvalid { rChild = join.children[1] } else { - rChild, err = a.tryToPushDownAgg(agg, rightAggFuncs, rightGbyCols, join, 1, agg.aggHints, agg.blockOffset, opt) + rChild, err = a.tryToPushDownAgg(agg, rightAggFuncs, rightGbyCols, join, 1, agg.aggHints, agg.SelectBlockOffset(), opt) if err != nil { return nil, err } @@ -497,7 +497,7 @@ func (a *aggregationPushDownSolver) aggPushDown(p LogicalPlan, opt *logicalOptim if leftInvalid { lChild = join.children[0] } else { - lChild, err = a.tryToPushDownAgg(agg, leftAggFuncs, leftGbyCols, join, 0, agg.aggHints, agg.blockOffset, opt) + lChild, err = a.tryToPushDownAgg(agg, leftAggFuncs, leftGbyCols, join, 0, agg.aggHints, agg.SelectBlockOffset(), opt) if err != nil { return nil, err } diff --git a/planner/core/rule_aggregation_skew_rewrite.go b/planner/core/rule_aggregation_skew_rewrite.go index e82e6833eb189..577533f152bbb 100644 --- a/planner/core/rule_aggregation_skew_rewrite.go +++ b/planner/core/rule_aggregation_skew_rewrite.go @@ -119,7 +119,7 @@ func (a *skewDistinctAggRewriter) rewriteSkewDistinctAgg(agg *LogicalAggregation } for _, arg := range aggFunc.Args { - firstRow, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncFirstRow, + firstRow, err := aggregation.NewAggFuncDesc(agg.SCtx(), ast.AggFuncFirstRow, []expression.Expression{arg}, false) if err != nil { return nil @@ -147,7 +147,7 @@ func (a *skewDistinctAggRewriter) rewriteSkewDistinctAgg(agg *LogicalAggregation } } else { aggCol = &expression.Column{ - UniqueID: agg.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: agg.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: newAggFunc.RetTp, } } @@ -155,14 +155,14 @@ func (a *skewDistinctAggRewriter) rewriteSkewDistinctAgg(agg *LogicalAggregation if newAggFunc.Name == ast.AggFuncCount { cntIndexes = append(cntIndexes, i) - sumAggFunc, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncSum, + sumAggFunc, err := aggregation.NewAggFuncDesc(agg.SCtx(), ast.AggFuncSum, []expression.Expression{aggCol}, false) if err != nil { return nil } topAggFuncs = append(topAggFuncs, sumAggFunc) topAggSchema.Columns[i] = &expression.Column{ - UniqueID: agg.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: agg.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: sumAggFunc.RetTp, } } else { @@ -179,7 +179,7 @@ func (a *skewDistinctAggRewriter) rewriteSkewDistinctAgg(agg *LogicalAggregation // SELECT count(DISTINCT a) FROM t GROUP BY b; // column b is not in the output schema, we have to add it to the bottom agg schema if firstRowCols.Has(int(col.UniqueID)) { - firstRow, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncFirstRow, + firstRow, err := aggregation.NewAggFuncDesc(agg.SCtx(), ast.AggFuncFirstRow, []expression.Expression{col}, false) if err != nil { return nil @@ -194,7 +194,7 @@ func (a *skewDistinctAggRewriter) rewriteSkewDistinctAgg(agg *LogicalAggregation AggFuncs: bottomAggFuncs, GroupByItems: bottomAggGroupbyItems, aggHints: agg.aggHints, - }.Init(agg.ctx, agg.blockOffset) + }.Init(agg.SCtx(), agg.SelectBlockOffset()) bottomAgg.SetChildren(agg.children...) bottomAgg.SetSchema(bottomAggSchema) @@ -202,7 +202,7 @@ func (a *skewDistinctAggRewriter) rewriteSkewDistinctAgg(agg *LogicalAggregation AggFuncs: topAggFuncs, GroupByItems: agg.GroupByItems, aggHints: agg.aggHints, - }.Init(agg.ctx, agg.blockOffset) + }.Init(agg.SCtx(), agg.SelectBlockOffset()) topAgg.SetChildren(bottomAgg) topAgg.SetSchema(topAggSchema) @@ -215,7 +215,7 @@ func (a *skewDistinctAggRewriter) rewriteSkewDistinctAgg(agg *LogicalAggregation // we have to return a project operator that casts decimal to bigint proj := LogicalProjection{ Exprs: make([]expression.Expression, 0, len(agg.AggFuncs)), - }.Init(agg.ctx, agg.blockOffset) + }.Init(agg.SCtx(), agg.SelectBlockOffset()) for _, column := range topAggSchema.Columns { proj.Exprs = append(proj.Exprs, column.Clone()) } @@ -225,7 +225,7 @@ func (a *skewDistinctAggRewriter) rewriteSkewDistinctAgg(agg *LogicalAggregation exprType := proj.Exprs[index].GetType() targetType := agg.schema.Columns[index].GetType() if !exprType.Equal(targetType) { - proj.Exprs[index] = expression.BuildCastFunction(agg.ctx, proj.Exprs[index], targetType) + proj.Exprs[index] = expression.BuildCastFunction(agg.SCtx(), proj.Exprs[index], targetType) } } proj.SetSchema(agg.schema.Clone()) diff --git a/planner/core/rule_build_key_info.go b/planner/core/rule_build_key_info.go index 5b2f5c4ad4580..bd957174c4296 100644 --- a/planner/core/rule_build_key_info.go +++ b/planner/core/rule_build_key_info.go @@ -152,7 +152,7 @@ func (p *LogicalProjection) buildSchemaByExprs(selfSchema *expression.Schema) *e } else { // If the expression is not a column, we add a column to occupy the position. schema.Append(&expression.Column{ - UniqueID: p.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: p.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: expr.GetType(), }) } @@ -201,13 +201,13 @@ func (p *LogicalJoin) BuildKeyInfo(selfSchema *expression.Schema, childSchema [] ln := expr.GetArgs()[0].(*expression.Column) rn := expr.GetArgs()[1].(*expression.Column) for _, key := range childSchema[0].Keys { - if len(key) == 1 && key[0].Equal(p.ctx, ln) { + if len(key) == 1 && key[0].Equal(p.SCtx(), ln) { lOk = true break } } for _, key := range childSchema[1].Keys { - if len(key) == 1 && key[0].Equal(p.ctx, rn) { + if len(key) == 1 && key[0].Equal(p.SCtx(), rn) { rOk = true break } @@ -271,11 +271,11 @@ func (ds *DataSource) BuildKeyInfo(selfSchema *expression.Schema, _ []*expressio var latestIndexes map[int64]*model.IndexInfo var changed bool var err error - check := ds.ctx.GetSessionVars().IsIsolation(ast.ReadCommitted) || ds.isForUpdateRead - check = check && ds.ctx.GetSessionVars().ConnectionID > 0 + check := ds.SCtx().GetSessionVars().IsIsolation(ast.ReadCommitted) || ds.isForUpdateRead + check = check && ds.SCtx().GetSessionVars().ConnectionID > 0 // we should check index valid while forUpdateRead, see detail in https://github.com/pingcap/tidb/pull/22152 if check { - latestIndexes, changed, err = getLatestIndexInfo(ds.ctx, ds.table.Meta().ID, 0) + latestIndexes, changed, err = getLatestIndexInfo(ds.SCtx(), ds.table.Meta().ID, 0) if err != nil { return } diff --git a/planner/core/rule_column_pruning.go b/planner/core/rule_column_pruning.go index 38348647eacb5..a987160205e75 100644 --- a/planner/core/rule_column_pruning.go +++ b/planner/core/rule_column_pruning.go @@ -157,16 +157,16 @@ func (la *LogicalAggregation) PruneColumns(parentUsedCols []*expression.Column, var err error var newAgg *aggregation.AggFuncDesc if allFirstRow { - newAgg, err = aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncFirstRow, []expression.Expression{expression.NewOne()}, false) + newAgg, err = aggregation.NewAggFuncDesc(la.SCtx(), ast.AggFuncFirstRow, []expression.Expression{expression.NewOne()}, false) } else { - newAgg, err = aggregation.NewAggFuncDesc(la.ctx, ast.AggFuncCount, []expression.Expression{expression.NewOne()}, false) + newAgg, err = aggregation.NewAggFuncDesc(la.SCtx(), ast.AggFuncCount, []expression.Expression{expression.NewOne()}, false) } if err != nil { return err } la.AggFuncs = append(la.AggFuncs, newAgg) col := &expression.Column{ - UniqueID: la.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: la.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: newAgg.RetTp, } la.schema.Columns = append(la.schema.Columns, col) @@ -301,7 +301,7 @@ func (p *LogicalUnionAll) PruneColumns(parentUsedCols []*expression.Column, opt for j, col := range schema.Columns { exprs[j] = col } - proj := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(p.ctx, p.blockOffset) + proj := LogicalProjection{Exprs: exprs, AvoidColumnEvaluator: true}.Init(p.SCtx(), p.SelectBlockOffset()) proj.SetSchema(schema) proj.SetChildren(child) @@ -603,7 +603,7 @@ func addConstOneForEmptyProjection(p LogicalPlan) { constOne := expression.NewOne() proj.schema.Append(&expression.Column{ - UniqueID: proj.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: proj.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: constOne.GetType(), }) proj.Exprs = append(proj.Exprs, &expression.Constant{ diff --git a/planner/core/rule_decorrelate.go b/planner/core/rule_decorrelate.go index 126bb1f027a4f..000a1a4acb291 100644 --- a/planner/core/rule_decorrelate.go +++ b/planner/core/rule_decorrelate.go @@ -46,7 +46,7 @@ func (la *LogicalAggregation) canPullUp() bool { } for _, f := range la.AggFuncs { for _, arg := range f.Args { - expr := expression.EvaluateExprWithNull(la.ctx, la.children[0].Schema(), arg) + expr := expression.EvaluateExprWithNull(la.SCtx(), la.children[0].Schema(), arg) if con, ok := expr.(*expression.Constant); !ok || !con.Value.IsNull() { return false } @@ -69,7 +69,7 @@ func (la *LogicalApply) deCorColFromEqExpr(expr expression.Expression) expressio return nil } // We should make sure that the equal condition's left side is the join's left join key, right is the right key. - return expression.NewFunctionInternal(la.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), ret, col) + return expression.NewFunctionInternal(la.SCtx(), ast.EQ, types.NewFieldType(mysql.TypeTiny), ret, col) } } if corCol, lOk := sf.GetArgs()[0].(*expression.CorrelatedColumn); lOk { @@ -79,7 +79,7 @@ func (la *LogicalApply) deCorColFromEqExpr(expr expression.Expression) expressio return nil } // We should make sure that the equal condition's left side is the join's left join key, right is the right key. - return expression.NewFunctionInternal(la.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), ret, col) + return expression.NewFunctionInternal(la.SCtx(), ast.EQ, types.NewFieldType(mysql.TypeTiny), ret, col) } } return nil @@ -202,7 +202,7 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo // If the inner plan is non-correlated, the apply will be simplified to join. join := &apply.LogicalJoin join.self = join - join.tp = plancodec.TypeJoin + join.SetTP(plancodec.TypeJoin) p = join appendApplySimplifiedTraceStep(apply, join, opt) } else if apply.NoDecorrelate { @@ -311,7 +311,7 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo outerColsInSchema := make([]*expression.Column, 0, outerPlan.Schema().Len()) for i, col := range outerPlan.Schema().Columns { - first, err := aggregation.NewAggFuncDesc(agg.ctx, ast.AggFuncFirstRow, []expression.Expression{col}, false) + first, err := aggregation.NewAggFuncDesc(agg.SCtx(), ast.AggFuncFirstRow, []expression.Expression{col}, false) if err != nil { return nil, err } @@ -341,7 +341,7 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo aggArgs = append(aggArgs, expr) } } - desc, err := aggregation.NewAggFuncDesc(agg.ctx, agg.AggFuncs[i].Name, aggArgs, agg.AggFuncs[i].HasDistinct) + desc, err := aggregation.NewAggFuncDesc(agg.SCtx(), agg.AggFuncs[i].Name, aggArgs, agg.AggFuncs[i].HasDistinct) if err != nil { return nil, err } @@ -389,7 +389,7 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo clonedCol := eqCond.GetArgs()[1].(*expression.Column) // If the join key is not in the aggregation's schema, add first row function. if agg.schema.ColumnIndex(eqCond.GetArgs()[1].(*expression.Column)) == -1 { - newFunc, err := aggregation.NewAggFuncDesc(apply.ctx, ast.AggFuncFirstRow, []expression.Expression{clonedCol}, false) + newFunc, err := aggregation.NewAggFuncDesc(apply.SCtx(), ast.AggFuncFirstRow, []expression.Expression{clonedCol}, false) if err != nil { return nil, err } @@ -413,12 +413,12 @@ func (s *decorrelateSolver) optimize(ctx context.Context, p LogicalPlan, opt *lo defaultValueMap := s.aggDefaultValueMap(agg) // We should use it directly, rather than building a projection. if len(defaultValueMap) > 0 { - proj := LogicalProjection{}.Init(agg.ctx, agg.blockOffset) + proj := LogicalProjection{}.Init(agg.SCtx(), agg.SelectBlockOffset()) proj.SetSchema(apply.schema) proj.Exprs = expression.Column2Exprs(apply.schema.Columns) for i, val := range defaultValueMap { pos := proj.schema.ColumnIndex(agg.schema.Columns[i]) - ifNullFunc := expression.NewFunctionInternal(agg.ctx, ast.Ifnull, types.NewFieldType(mysql.TypeLonglong), agg.schema.Columns[i], val) + ifNullFunc := expression.NewFunctionInternal(agg.SCtx(), ast.Ifnull, types.NewFieldType(mysql.TypeLonglong), agg.schema.Columns[i], val) proj.Exprs[pos] = ifNullFunc } proj.SetChildren(apply) diff --git a/planner/core/rule_derive_topn_from_window.go b/planner/core/rule_derive_topn_from_window.go index 31901e45643c4..50d7d085ef2cf 100644 --- a/planner/core/rule_derive_topn_from_window.go +++ b/planner/core/rule_derive_topn_from_window.go @@ -91,7 +91,7 @@ func windowIsTopN(p *LogicalSelection) (bool, uint64) { // Check if filter on window function windowColumns := child.GetWindowResultColumns() - if len(windowColumns) != 1 || !(column.Equal(p.ctx, windowColumns[0])) { + if len(windowColumns) != 1 || !(column.Equal(p.SCtx(), windowColumns[0])) { return false, 0 } @@ -143,7 +143,7 @@ func (s *LogicalSelection) deriveTopN(opt *logicalOptimizeOp) LogicalPlan { byItems = append(byItems, &util.ByItems{Expr: col.Col, Desc: col.Desc}) } // Build derived Limit - derivedTopN := LogicalTopN{Count: limitValue, ByItems: byItems, PartitionBy: child.GetPartitionBy()}.Init(grandChild.ctx, grandChild.blockOffset) + derivedTopN := LogicalTopN{Count: limitValue, ByItems: byItems, PartitionBy: child.GetPartitionBy()}.Init(grandChild.SCtx(), grandChild.SelectBlockOffset()) derivedTopN.SetChildren(grandChild) /* return select->datasource->topN->window */ child.SetChildren(derivedTopN) diff --git a/planner/core/rule_eliminate_projection.go b/planner/core/rule_eliminate_projection.go index 76b7473c13bc0..d744092ae7968 100644 --- a/planner/core/rule_eliminate_projection.go +++ b/planner/core/rule_eliminate_projection.go @@ -80,7 +80,7 @@ func canProjectionBeEliminatedStrict(p *PhysicalProjection) bool { if p.Schema().Len() != child.Schema().Len() { return false } - for _, ref := range p.ctx.GetSessionVars().StmtCtx.ColRefFromUpdatePlan { + for _, ref := range p.SCtx().GetSessionVars().StmtCtx.ColRefFromUpdatePlan { for _, one := range p.Schema().Columns { if ref == one.UniqueID { return false @@ -203,7 +203,7 @@ func (pe *projectionEliminator) eliminate(p LogicalPlan, replace map[string]*exp resolveColumnAndReplace(dst, replace) } } - p.replaceExprColumns(replace) + p.ReplaceExprColumns(replace) if isProj { if child, ok := p.Children()[0].(*LogicalProjection); ok && !ExprsHasSideEffects(child.Exprs) { for i := range proj.Exprs { @@ -245,7 +245,8 @@ func ReplaceColumnOfExpr(expr expression.Expression, proj *LogicalProjection, sc return expr } -func (p *LogicalJoin) replaceExprColumns(replace map[string]*expression.Column) { +// ReplaceExprColumns implements LogicalPlan interface. +func (p *LogicalJoin) ReplaceExprColumns(replace map[string]*expression.Column) { for _, equalExpr := range p.EqualConditions { ResolveExprAndReplace(equalExpr, replace) } @@ -260,13 +261,15 @@ func (p *LogicalJoin) replaceExprColumns(replace map[string]*expression.Column) } } -func (p *LogicalProjection) replaceExprColumns(replace map[string]*expression.Column) { +// ReplaceExprColumns implements LogicalPlan interface. +func (p *LogicalProjection) ReplaceExprColumns(replace map[string]*expression.Column) { for _, expr := range p.Exprs { ResolveExprAndReplace(expr, replace) } } -func (la *LogicalAggregation) replaceExprColumns(replace map[string]*expression.Column) { +// ReplaceExprColumns implements LogicalPlan interface. +func (la *LogicalAggregation) ReplaceExprColumns(replace map[string]*expression.Column) { for _, agg := range la.AggFuncs { for _, aggExpr := range agg.Args { ResolveExprAndReplace(aggExpr, replace) @@ -277,14 +280,16 @@ func (la *LogicalAggregation) replaceExprColumns(replace map[string]*expression. } } -func (p *LogicalSelection) replaceExprColumns(replace map[string]*expression.Column) { +// ReplaceExprColumns implements LogicalPlan interface. +func (p *LogicalSelection) ReplaceExprColumns(replace map[string]*expression.Column) { for _, expr := range p.Conditions { ResolveExprAndReplace(expr, replace) } } -func (la *LogicalApply) replaceExprColumns(replace map[string]*expression.Column) { - la.LogicalJoin.replaceExprColumns(replace) +// ReplaceExprColumns implements LogicalPlan interface. +func (la *LogicalApply) ReplaceExprColumns(replace map[string]*expression.Column) { + la.LogicalJoin.ReplaceExprColumns(replace) for _, coCol := range la.CorCols { dst := replace[string(coCol.Column.HashCode(nil))] if dst != nil { @@ -293,19 +298,22 @@ func (la *LogicalApply) replaceExprColumns(replace map[string]*expression.Column } } -func (ls *LogicalSort) replaceExprColumns(replace map[string]*expression.Column) { +// ReplaceExprColumns implements LogicalPlan interface. +func (ls *LogicalSort) ReplaceExprColumns(replace map[string]*expression.Column) { for _, byItem := range ls.ByItems { ResolveExprAndReplace(byItem.Expr, replace) } } -func (lt *LogicalTopN) replaceExprColumns(replace map[string]*expression.Column) { +// ReplaceExprColumns implements LogicalPlan interface. +func (lt *LogicalTopN) ReplaceExprColumns(replace map[string]*expression.Column) { for _, byItem := range lt.ByItems { ResolveExprAndReplace(byItem.Expr, replace) } } -func (p *LogicalWindow) replaceExprColumns(replace map[string]*expression.Column) { +// ReplaceExprColumns implements LogicalPlan interface. +func (p *LogicalWindow) ReplaceExprColumns(replace map[string]*expression.Column) { for _, desc := range p.WindowFuncDescs { for _, arg := range desc.Args { ResolveExprAndReplace(arg, replace) diff --git a/planner/core/rule_inject_extra_projection.go b/planner/core/rule_inject_extra_projection.go index 9d5f86e41c0f6..edd972426fea7 100644 --- a/planner/core/rule_inject_extra_projection.go +++ b/planner/core/rule_inject_extra_projection.go @@ -90,7 +90,7 @@ func injectProjBelowUnion(un *PhysicalUnionAll) *PhysicalUnionAll { srcCol.Index = i srcType := srcCol.RetType if !srcType.Equal(dstType) || !(mysql.HasNotNullFlag(dstType.GetFlag()) == mysql.HasNotNullFlag(srcType.GetFlag())) { - exprs[i] = expression.BuildCastFunction4Union(un.ctx, srcCol, dstType) + exprs[i] = expression.BuildCastFunction4Union(un.SCtx(), srcCol, dstType) needChange = true } else { exprs[i] = srcCol @@ -99,7 +99,7 @@ func injectProjBelowUnion(un *PhysicalUnionAll) *PhysicalUnionAll { if needChange { proj := PhysicalProjection{ Exprs: exprs, - }.Init(un.ctx, ch.statsInfo(), 0) + }.Init(un.SCtx(), ch.StatsInfo(), 0) proj.SetSchema(un.schema.Clone()) proj.SetChildren(ch) un.children[i] = proj @@ -189,7 +189,7 @@ func InjectProjBelowAgg(aggPlan PhysicalPlan, aggFuncs []*aggregation.AggFuncDes proj := PhysicalProjection{ Exprs: projExprs, AvoidColumnEvaluator: false, - }.Init(aggPlan.SCtx(), child.statsInfo().ScaleByExpectCnt(prop.ExpectedCnt), aggPlan.SelectBlockOffset(), prop) + }.Init(aggPlan.SCtx(), child.StatsInfo().ScaleByExpectCnt(prop.ExpectedCnt), aggPlan.SelectBlockOffset(), prop) proj.SetSchema(expression.NewSchema(projSchemaCols...)) proj.SetChildren(child) @@ -223,7 +223,7 @@ func InjectProjBelowSort(p PhysicalPlan, orderByItems []*util.ByItems) PhysicalP topProj := PhysicalProjection{ Exprs: topProjExprs, AvoidColumnEvaluator: false, - }.Init(p.SCtx(), p.statsInfo(), p.SelectBlockOffset(), nil) + }.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), nil) topProj.SetSchema(p.Schema().Clone()) topProj.SetChildren(p) @@ -256,7 +256,7 @@ func InjectProjBelowSort(p PhysicalPlan, orderByItems []*util.ByItems) PhysicalP bottomProj := PhysicalProjection{ Exprs: bottomProjExprs, AvoidColumnEvaluator: false, - }.Init(p.SCtx(), childPlan.statsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) + }.Init(p.SCtx(), childPlan.StatsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) bottomProj.SetSchema(expression.NewSchema(bottomProjSchemaCols...)) bottomProj.SetChildren(childPlan) p.SetChildren(bottomProj) @@ -305,7 +305,7 @@ func TurnNominalSortIntoProj(p PhysicalPlan, onlyColumn bool, orderByItems []*ut bottomProj := PhysicalProjection{ Exprs: bottomProjExprs, AvoidColumnEvaluator: false, - }.Init(p.SCtx(), childPlan.statsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) + }.Init(p.SCtx(), childPlan.StatsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) bottomProj.SetSchema(expression.NewSchema(bottomProjSchemaCols...)) bottomProj.SetChildren(childPlan) @@ -318,7 +318,7 @@ func TurnNominalSortIntoProj(p PhysicalPlan, onlyColumn bool, orderByItems []*ut topProj := PhysicalProjection{ Exprs: topProjExprs, AvoidColumnEvaluator: false, - }.Init(p.SCtx(), childPlan.statsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) + }.Init(p.SCtx(), childPlan.StatsInfo().ScaleByExpectCnt(childProp.ExpectedCnt), p.SelectBlockOffset(), childProp) topProj.SetSchema(childPlan.Schema().Clone()) topProj.SetChildren(bottomProj) diff --git a/planner/core/rule_join_reorder.go b/planner/core/rule_join_reorder.go index 20ceabd975264..9376a81d2d874 100644 --- a/planner/core/rule_join_reorder.go +++ b/planner/core/rule_join_reorder.go @@ -481,7 +481,7 @@ func (s *baseSingleGroupJoinOrderSolver) generateJoinOrderNode(joinNodePlans []L // baseNodeCumCost calculate the cumulative cost of the node in the join group. func (s *baseSingleGroupJoinOrderSolver) baseNodeCumCost(groupNode LogicalPlan) float64 { - cost := groupNode.statsInfo().RowCount + cost := groupNode.StatsInfo().RowCount for _, child := range groupNode.Children() { cost += s.baseNodeCumCost(child) } @@ -643,7 +643,7 @@ func (s *baseSingleGroupJoinOrderSolver) setNewJoinWithHint(newJoin *LogicalJoin // calcJoinCumCost calculates the cumulative cost of the join node. func (*baseSingleGroupJoinOrderSolver) calcJoinCumCost(join LogicalPlan, lNode, rNode *jrNode) float64 { - return join.statsInfo().RowCount + lNode.cumCost + rNode.cumCost + return join.StatsInfo().RowCount + lNode.cumCost + rNode.cumCost } func (*joinReOrderSolver) name() string { @@ -771,16 +771,16 @@ func (t *joinReorderTrace) traceJoinReorder(p LogicalPlan) { return } if len(t.initial) > 0 { - t.final = allJoinOrderToString(extractJoinAndDataSource(p.buildPlanTrace())) + t.final = allJoinOrderToString(extractJoinAndDataSource(p.BuildPlanTrace())) return } - t.initial = allJoinOrderToString(extractJoinAndDataSource(p.buildPlanTrace())) + t.initial = allJoinOrderToString(extractJoinAndDataSource(p.BuildPlanTrace())) } func (t *joinReorderTrace) appendLogicalJoinCost(join LogicalPlan, cost float64) { if t == nil || t.opt == nil || t.opt.tracer == nil { return } - joinMapKey := allJoinOrderToString(extractJoinAndDataSource(join.buildPlanTrace())) + joinMapKey := allJoinOrderToString(extractJoinAndDataSource(join.BuildPlanTrace())) t.cost[joinMapKey] = cost } diff --git a/planner/core/rule_join_reorder_dp_test.go b/planner/core/rule_join_reorder_dp_test.go index bfe28fdffa7be..760941c552d50 100644 --- a/planner/core/rule_join_reorder_dp_test.go +++ b/planner/core/rule_join_reorder_dp_test.go @@ -41,8 +41,8 @@ func (mj mockLogicalJoin) init(ctx sessionctx.Context) *mockLogicalJoin { } func (mj *mockLogicalJoin) recursiveDeriveStats(_ [][]*expression.Column) (*property.StatsInfo, error) { - if mj.stats == nil { - mj.stats = mj.statsMap[mj.involvedNodeSet] + if mj.StatsInfo() == nil { + mj.SetStats(mj.statsMap[mj.involvedNodeSet]) } return mj.statsMap[mj.involvedNodeSet], nil } @@ -142,9 +142,9 @@ func newDataSource(ctx sessionctx.Context, name string, count int) LogicalPlan { UniqueID: ctx.GetSessionVars().PlanColumnID.Add(1), RetType: types.NewFieldType(mysql.TypeLonglong), }) - ds.stats = &property.StatsInfo{ + ds.SetStats(&property.StatsInfo{ RowCount: float64(count), - } + }) return ds } diff --git a/planner/core/rule_max_min_eliminate.go b/planner/core/rule_max_min_eliminate.go index 08fec06d7384a..2ea1a35e4cb2b 100644 --- a/planner/core/rule_max_min_eliminate.go +++ b/planner/core/rule_max_min_eliminate.go @@ -73,7 +73,7 @@ func (a *maxMinEliminator) checkColCanUseIndex(plan LogicalPlan, col *expression // we only need to check if all of the conditions can be pushed down as accessConds // and `col` is the handle column. if p.handleCols != nil && col.Equal(nil, p.handleCols.GetCol(0)) { - if _, filterConds := ranger.DetachCondsForColumn(p.ctx, conditions, col); len(filterConds) != 0 { + if _, filterConds := ranger.DetachCondsForColumn(p.SCtx(), conditions, col); len(filterConds) != 0 { return false } return true @@ -85,7 +85,7 @@ func (a *maxMinEliminator) checkColCanUseIndex(plan LogicalPlan, col *expression } // 1. whether all of the conditions can be pushed down as accessConds. // 2. whether the AccessPath can satisfy the order property of `col` with these accessConds. - result, err := ranger.DetachCondAndBuildRangeForIndex(p.ctx, conditions, indexCols, indexColLen, p.ctx.GetSessionVars().RangeMaxSize) + result, err := ranger.DetachCondAndBuildRangeForIndex(p.SCtx(), conditions, indexCols, indexColLen, p.SCtx().GetSessionVars().RangeMaxSize) if err != nil || len(result.RemainedConds) != 0 { continue } @@ -109,14 +109,14 @@ func (a *maxMinEliminator) cloneSubPlans(plan LogicalPlan) LogicalPlan { case *LogicalSelection: newConditions := make([]expression.Expression, len(p.Conditions)) copy(newConditions, p.Conditions) - sel := LogicalSelection{Conditions: newConditions}.Init(p.ctx, p.blockOffset) + sel := LogicalSelection{Conditions: newConditions}.Init(p.SCtx(), p.SelectBlockOffset()) sel.SetChildren(a.cloneSubPlans(p.children[0])) return sel case *DataSource: // Quick clone a DataSource. // ReadOnly fields uses a shallow copy, while the fields which will be overwritten must use a deep copy. newDs := *p - newDs.baseLogicalPlan = newBaseLogicalPlan(p.ctx, p.tp, &newDs, p.blockOffset) + newDs.baseLogicalPlan = newBaseLogicalPlan(p.SCtx(), p.TP(), &newDs, p.SelectBlockOffset()) newDs.schema = p.schema.Clone() newDs.Columns = make([]*model.ColumnInfo, len(p.Columns)) copy(newDs.Columns, p.Columns) @@ -151,7 +151,7 @@ func (a *maxMinEliminator) splitAggFuncAndCheckIndices(agg *LogicalAggregation, aggs = make([]*LogicalAggregation, 0, len(agg.AggFuncs)) // we can split the aggregation only if all of the aggFuncs pass the check. for i, f := range agg.AggFuncs { - newAgg := LogicalAggregation{AggFuncs: []*aggregation.AggFuncDesc{f}}.Init(agg.ctx, agg.blockOffset) + newAgg := LogicalAggregation{AggFuncs: []*aggregation.AggFuncDesc{f}}.Init(agg.SCtx(), agg.SelectBlockOffset()) newAgg.SetChildren(a.cloneSubPlans(agg.children[0])) newAgg.schema = expression.NewSchema(agg.schema.Columns[i]) if err := newAgg.PruneColumns([]*expression.Column{newAgg.schema.Columns[0]}, opt); err != nil { @@ -174,7 +174,7 @@ func (*maxMinEliminator) eliminateSingleMaxMin(agg *LogicalAggregation, opt *log if len(expression.ExtractColumns(f.Args[0])) > 0 { // If it can be NULL, we need to filter NULL out first. if !mysql.HasNotNullFlag(f.Args[0].GetType().GetFlag()) { - sel = LogicalSelection{}.Init(ctx, agg.blockOffset) + sel = LogicalSelection{}.Init(ctx, agg.SelectBlockOffset()) isNullFunc := expression.NewFunctionInternal(ctx, ast.IsNull, types.NewFieldType(mysql.TypeTiny), f.Args[0]) notNullFunc := expression.NewFunctionInternal(ctx, ast.UnaryNot, types.NewFieldType(mysql.TypeTiny), isNullFunc) sel.Conditions = []expression.Expression{notNullFunc} @@ -186,14 +186,14 @@ func (*maxMinEliminator) eliminateSingleMaxMin(agg *LogicalAggregation, opt *log // For max function, the sort order should be desc. desc := f.Name == ast.AggFuncMax // Compose Sort operator. - sort = LogicalSort{}.Init(ctx, agg.blockOffset) + sort = LogicalSort{}.Init(ctx, agg.SelectBlockOffset()) sort.ByItems = append(sort.ByItems, &util.ByItems{Expr: f.Args[0], Desc: desc}) sort.SetChildren(child) child = sort } // Compose Limit operator. - li := LogicalLimit{Count: 1}.Init(ctx, agg.blockOffset) + li := LogicalLimit{Count: 1}.Init(ctx, agg.SelectBlockOffset()) li.SetChildren(child) // If no data in the child, we need to return NULL instead of empty. This cannot be done by sort and limit themselves. diff --git a/planner/core/rule_partition_processor.go b/planner/core/rule_partition_processor.go index 363835ae58c4a..d39128a73836e 100644 --- a/planner/core/rule_partition_processor.go +++ b/planner/core/rule_partition_processor.go @@ -84,7 +84,7 @@ func (s *partitionProcessor) rewriteDataSource(lp LogicalPlan, opt *logicalOptim us := LogicalUnionScan{ conditions: p.conditions, handleCols: p.handleCols, - }.Init(ua.ctx, ua.blockOffset) + }.Init(ua.SCtx(), ua.SelectBlockOffset()) us.SetChildren(child) children = append(children, us) } @@ -467,7 +467,7 @@ func (s *partitionProcessor) processHashOrKeyPartition(ds *DataSource, pi *model if used != nil { return s.makeUnionAllChildren(ds, pi, convertToRangeOr(used, pi), opt) } - tableDual := LogicalTableDual{RowCount: 0}.Init(ds.SCtx(), ds.blockOffset) + tableDual := LogicalTableDual{RowCount: 0}.Init(ds.SCtx(), ds.SelectBlockOffset()) tableDual.schema = ds.Schema() appendNoPartitionChildTraceStep(ds, tableDual, opt) return tableDual, nil @@ -759,7 +759,7 @@ func (s *partitionProcessor) prune(ds *DataSource, opt *logicalOptimizeOp) (Logi // like 'not (a != 1)' would not be handled so we need to convert it to 'a = 1', which can be handled when building range. // TODO: there may be a better way to push down Not once for all. for i, cond := range ds.allConds { - ds.allConds[i] = expression.PushDownNot(ds.ctx, cond) + ds.allConds[i] = expression.PushDownNot(ds.SCtx(), cond) } // Try to locate partition directly for hash partition. switch pi.Type { @@ -978,7 +978,7 @@ func (s *partitionProcessor) pruneRangePartition(ctx sessionctx.Context, pi *mod } func (s *partitionProcessor) processRangePartition(ds *DataSource, pi *model.PartitionInfo, opt *logicalOptimizeOp) (LogicalPlan, error) { - used, err := s.pruneRangePartition(ds.ctx, pi, ds.table.(table.PartitionedTable), ds.allConds, ds.TblCols, ds.names) + used, err := s.pruneRangePartition(ds.SCtx(), pi, ds.table.(table.PartitionedTable), ds.allConds, ds.TblCols, ds.names) if err != nil { return nil, err } @@ -993,7 +993,7 @@ func (s *partitionProcessor) processListPartition(ds *DataSource, pi *model.Part if used != nil { return s.makeUnionAllChildren(ds, pi, convertToRangeOr(used, pi), opt) } - tableDual := LogicalTableDual{RowCount: 0}.Init(ds.SCtx(), ds.blockOffset) + tableDual := LogicalTableDual{RowCount: 0}.Init(ds.SCtx(), ds.SelectBlockOffset()) tableDual.schema = ds.Schema() appendNoPartitionChildTraceStep(ds, tableDual, opt) return tableDual, nil @@ -1590,12 +1590,12 @@ func pruneUseBinarySearch(lessThan lessThanDataInt, data dataForPrune, unsigned func (*partitionProcessor) resolveAccessPaths(ds *DataSource) error { possiblePaths, err := getPossibleAccessPaths( - ds.ctx, &tableHintInfo{indexMergeHintList: ds.indexMergeHints, indexHintList: ds.IndexHints}, + ds.SCtx(), &tableHintInfo{indexMergeHintList: ds.indexMergeHints, indexHintList: ds.IndexHints}, ds.astIndexHints, ds.table, ds.DBName, ds.tableInfo.Name, ds.isForUpdateRead, true) if err != nil { return err } - possiblePaths, err = filterPathByIsolationRead(ds.ctx, possiblePaths, ds.tableInfo.Name, ds.DBName) + possiblePaths, err = filterPathByIsolationRead(ds.SCtx(), possiblePaths, ds.tableInfo.Name, ds.DBName) if err != nil { return err } @@ -1662,7 +1662,7 @@ func (s *partitionProcessor) resolveOptimizeHint(ds *DataSource, partitionName m } } if ds.preferStoreType&preferTiFlash != 0 && ds.preferStoreType&preferTiKV != 0 { - ds.ctx.GetSessionVars().StmtCtx.AppendWarning( + ds.SCtx().GetSessionVars().StmtCtx.AppendWarning( errors.New("hint `read_from_storage` has conflict storage type for the partition " + partitionName.L)) } @@ -1691,16 +1691,16 @@ func appendWarnForUnknownPartitions(ctx sessionctx.Context, hintName string, unk func (*partitionProcessor) checkHintsApplicable(ds *DataSource, partitionSet set.StringSet) { for _, idxHint := range ds.IndexHints { unknownPartitions := checkTableHintsApplicableForPartition(idxHint.partitions, partitionSet) - appendWarnForUnknownPartitions(ds.ctx, restore2IndexHint(idxHint.hintTypeString(), idxHint), unknownPartitions) + appendWarnForUnknownPartitions(ds.SCtx(), restore2IndexHint(idxHint.hintTypeString(), idxHint), unknownPartitions) } for _, idxMergeHint := range ds.indexMergeHints { unknownPartitions := checkTableHintsApplicableForPartition(idxMergeHint.partitions, partitionSet) - appendWarnForUnknownPartitions(ds.ctx, restore2IndexHint(HintIndexMerge, idxMergeHint), unknownPartitions) + appendWarnForUnknownPartitions(ds.SCtx(), restore2IndexHint(HintIndexMerge, idxMergeHint), unknownPartitions) } unknownPartitions := checkTableHintsApplicableForPartition(ds.preferPartitions[preferTiKV], partitionSet) unknownPartitions = append(unknownPartitions, checkTableHintsApplicableForPartition(ds.preferPartitions[preferTiFlash], partitionSet)...) - appendWarnForUnknownPartitions(ds.ctx, HintReadFromStorage, unknownPartitions) + appendWarnForUnknownPartitions(ds.SCtx(), HintReadFromStorage, unknownPartitions) } func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.PartitionInfo, or partitionRangeOR, opt *logicalOptimizeOp) (LogicalPlan, error) { @@ -1717,7 +1717,7 @@ func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.Part } // Not a deep copy. newDataSource := *ds - newDataSource.baseLogicalPlan = newBaseLogicalPlan(ds.SCtx(), plancodec.TypeTableScan, &newDataSource, ds.blockOffset) + newDataSource.baseLogicalPlan = newBaseLogicalPlan(ds.SCtx(), plancodec.TypeTableScan, &newDataSource, ds.SelectBlockOffset()) newDataSource.schema = ds.schema.Clone() newDataSource.Columns = make([]*model.ColumnInfo, len(ds.Columns)) copy(newDataSource.Columns, ds.Columns) @@ -1727,7 +1727,7 @@ func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.Part // There are many expression nodes in the plan tree use the original datasource // id as FromID. So we set the id of the newDataSource with the original one to // avoid traversing the whole plan tree to update the references. - newDataSource.id = ds.id + newDataSource.SetID(ds.ID()) err := s.resolveOptimizeHint(&newDataSource, pi.Definitions[i].Name) partitionNameSet.Insert(pi.Definitions[i].Name.L) if err != nil { @@ -1741,7 +1741,7 @@ func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.Part if len(children) == 0 { // No result after table pruning. - tableDual := LogicalTableDual{RowCount: 0}.Init(ds.SCtx(), ds.blockOffset) + tableDual := LogicalTableDual{RowCount: 0}.Init(ds.SCtx(), ds.SelectBlockOffset()) tableDual.schema = ds.Schema() appendMakeUnionAllChildrenTranceStep(ds, usedDefinition, tableDual, children, opt) return tableDual, nil @@ -1751,7 +1751,7 @@ func (s *partitionProcessor) makeUnionAllChildren(ds *DataSource, pi *model.Part appendMakeUnionAllChildrenTranceStep(ds, usedDefinition, children[0], children, opt) return children[0], nil } - unionAll := LogicalPartitionUnionAll{}.Init(ds.SCtx(), ds.blockOffset) + unionAll := LogicalPartitionUnionAll{}.Init(ds.SCtx(), ds.SelectBlockOffset()) unionAll.SetChildren(children...) unionAll.SetSchema(ds.schema.Clone()) appendMakeUnionAllChildrenTranceStep(ds, usedDefinition, unionAll, children, opt) @@ -1955,7 +1955,7 @@ func appendMakeUnionAllChildrenTranceStep(origin *DataSource, usedMap map[int64] }) if len(children) == 1 { newDS := plan.(*DataSource) - newDS.id = origin.SCtx().GetSessionVars().AllocNewPlanID() + newDS.SetID(origin.SCtx().GetSessionVars().AllocNewPlanID()) action = func() string { return fmt.Sprintf("%v_%v becomes %s_%v", origin.TP(), origin.ID(), newDS.TP(), newDS.ID()) } @@ -1971,7 +1971,7 @@ func appendMakeUnionAllChildrenTranceStep(origin *DataSource, usedMap map[int64] buffer.WriteString(",") } newDS := child.(*DataSource) - newDS.id = origin.SCtx().GetSessionVars().AllocNewPlanID() + newDS.SetID(origin.SCtx().GetSessionVars().AllocNewPlanID()) fmt.Fprintf(buffer, "%s_%v", child.TP(), newDS.ID()) } buffer.WriteString("]") diff --git a/planner/core/rule_predicate_push_down.go b/planner/core/rule_predicate_push_down.go index c9362744cbe15..1c5f468409359 100644 --- a/planner/core/rule_predicate_push_down.go +++ b/planner/core/rule_predicate_push_down.go @@ -108,7 +108,7 @@ func (p *LogicalSelection) PredicatePushDown(predicates []expression.Expression, retConditions, child = p.children[0].PredicatePushDown(append(canBePushDown, predicates...), opt) retConditions = append(retConditions, canNotBePushDown...) if len(retConditions) > 0 { - p.Conditions = expression.PropagateConstant(p.ctx, retConditions) + p.Conditions = expression.PropagateConstant(p.SCtx(), retConditions) // Return table dual when filter is constant false or null. dual := Conds2TableDual(p, p.Conditions) if dual != nil { @@ -132,13 +132,13 @@ func (p *LogicalUnionScan) PredicatePushDown(predicates []expression.Expression, // PredicatePushDown implements LogicalPlan PredicatePushDown interface. func (ds *DataSource) PredicatePushDown(predicates []expression.Expression, opt *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { - predicates = expression.PropagateConstant(ds.ctx, predicates) + predicates = expression.PropagateConstant(ds.SCtx(), predicates) predicates = DeleteTrueExprs(ds, predicates) // Add tidb_shard() prefix to the condtion for shard index in some scenarios // TODO: remove it to the place building logical plan - predicates = ds.AddPrefix4ShardIndexes(ds.ctx, predicates) + predicates = ds.AddPrefix4ShardIndexes(ds.SCtx(), predicates) ds.allConds = predicates - ds.pushedDownConds, predicates = expression.PushDownExprs(ds.ctx.GetSessionVars().StmtCtx, predicates, ds.ctx.GetClient(), kv.UnSpecified) + ds.pushedDownConds, predicates = expression.PushDownExprs(ds.SCtx().GetSessionVars().StmtCtx, predicates, ds.SCtx().GetClient(), kv.UnSpecified) appendDataSourcePredicatePushDownTraceStep(ds, opt) return predicates, ds } @@ -162,7 +162,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt return ret, dual } // Handle where conditions - predicates = expression.ExtractFiltersFromDNFs(p.ctx, predicates) + predicates = expression.ExtractFiltersFromDNFs(p.SCtx(), predicates) // Only derive left where condition, because right where condition cannot be pushed down equalCond, leftPushCond, rightPushCond, otherCond = p.extractOnCondition(predicates, true, false) leftCond = leftPushCond @@ -181,7 +181,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt return ret, dual } // Handle where conditions - predicates = expression.ExtractFiltersFromDNFs(p.ctx, predicates) + predicates = expression.ExtractFiltersFromDNFs(p.SCtx(), predicates) // Only derive right where condition, because left where condition cannot be pushed down equalCond, leftPushCond, rightPushCond, otherCond = p.extractOnCondition(predicates, false, true) rightCond = rightPushCond @@ -199,8 +199,8 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt tempCond = append(tempCond, expression.ScalarFuncs2Exprs(p.EqualConditions)...) tempCond = append(tempCond, p.OtherConditions...) tempCond = append(tempCond, predicates...) - tempCond = expression.ExtractFiltersFromDNFs(p.ctx, tempCond) - tempCond = expression.PropagateConstant(p.ctx, tempCond) + tempCond = expression.ExtractFiltersFromDNFs(p.SCtx(), tempCond) + tempCond = expression.PropagateConstant(p.SCtx(), tempCond) // Return table dual when filter is constant false or null. dual := Conds2TableDual(p, tempCond) if dual != nil { @@ -215,7 +215,7 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt leftCond = leftPushCond rightCond = rightPushCond case AntiSemiJoin: - predicates = expression.PropagateConstant(p.ctx, predicates) + predicates = expression.PropagateConstant(p.SCtx(), predicates) // Return table dual when filter is constant false or null. dual := Conds2TableDual(p, predicates) if dual != nil { @@ -233,8 +233,8 @@ func (p *LogicalJoin) PredicatePushDown(predicates []expression.Expression, opt rightCond = append(p.RightConditions, rightPushCond...) p.RightConditions = nil } - leftCond = expression.RemoveDupExprs(p.ctx, leftCond) - rightCond = expression.RemoveDupExprs(p.ctx, rightCond) + leftCond = expression.RemoveDupExprs(p.SCtx(), leftCond) + rightCond = expression.RemoveDupExprs(p.SCtx(), rightCond) leftRet, lCh := p.children[0].PredicatePushDown(leftCond, opt) rightRet, rCh := p.children[1].PredicatePushDown(rightCond, opt) addSelection(p, lCh, leftRet, 0, opt) @@ -304,7 +304,7 @@ func (p *LogicalJoin) updateEQCond() { if rProj != nil { rKey = rProj.appendExpr(rKey) } - eqCond := expression.NewFunctionInternal(p.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), lKey, rKey) + eqCond := expression.NewFunctionInternal(p.SCtx(), ast.EQ, types.NewFieldType(mysql.TypeTiny), lKey, rKey) if isNA { p.NAEQConditions = append(p.NAEQConditions, eqCond.(*expression.ScalarFunction)) } else { @@ -353,7 +353,7 @@ func (p *LogicalProjection) appendExpr(expr expression.Expression) *expression.C p.Exprs = append(p.Exprs, expr) col := &expression.Column{ - UniqueID: p.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: p.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: expr.GetType().Clone(), } col.SetCoercibility(expr.Coercibility()) @@ -372,7 +372,7 @@ func (p *LogicalJoin) getProj(idx int) *LogicalProjection { if ok { return proj } - proj = LogicalProjection{Exprs: make([]expression.Expression, 0, child.Schema().Len())}.Init(p.ctx, child.SelectBlockOffset()) + proj = LogicalProjection{Exprs: make([]expression.Expression, 0, child.Schema().Len())}.Init(p.SCtx(), child.SelectBlockOffset()) for _, col := range child.Schema().Columns { proj.Exprs = append(proj.Exprs, col) } @@ -412,7 +412,7 @@ func simplifyOuterJoin(p *LogicalJoin, predicates []expression.Expression) { if expression.ExprFromSchema(expr, outerTable.Schema()) { continue } - isOk := isNullRejected(p.ctx, innerTable.Schema(), expr) + isOk := isNullRejected(p.SCtx(), innerTable.Schema(), expr) if isOk { canBeSimplified = true break @@ -538,10 +538,10 @@ func (la *LogicalAggregation) pushDownCNFPredicatesForAggregation(cond expressio for _, item := range subCNFItem { condsToPushForItem, retForItem := la.pushDownDNFPredicatesForAggregation(item, groupByColumns, exprsOriginal) if len(condsToPushForItem) > 0 { - condsToPush = append(condsToPush, expression.ComposeDNFCondition(la.ctx, condsToPushForItem...)) + condsToPush = append(condsToPush, expression.ComposeDNFCondition(la.SCtx(), condsToPushForItem...)) } if len(retForItem) > 0 { - ret = append(ret, expression.ComposeDNFCondition(la.ctx, retForItem...)) + ret = append(ret, expression.ComposeDNFCondition(la.SCtx(), retForItem...)) } } return condsToPush, ret @@ -565,16 +565,16 @@ func (la *LogicalAggregation) pushDownDNFPredicatesForAggregation(cond expressio if len(condsToPushForItem) <= 0 { return nil, []expression.Expression{cond} } - condsToPush = append(condsToPush, expression.ComposeCNFCondition(la.ctx, condsToPushForItem...)) + condsToPush = append(condsToPush, expression.ComposeCNFCondition(la.SCtx(), condsToPushForItem...)) if len(retForItem) > 0 { - ret = append(ret, expression.ComposeCNFCondition(la.ctx, retForItem...)) + ret = append(ret, expression.ComposeCNFCondition(la.SCtx(), retForItem...)) } } if len(ret) == 0 { // All the condition can be pushed down. return []expression.Expression{cond}, nil } - dnfPushDownCond := expression.ComposeDNFCondition(la.ctx, condsToPush...) + dnfPushDownCond := expression.ComposeDNFCondition(la.SCtx(), condsToPush...) // Some condition can't be pushed down, we need to keep all the condition. return []expression.Expression{dnfPushDownCond}, []expression.Expression{cond} } @@ -744,7 +744,7 @@ func (p *LogicalJoin) outerJoinPropConst(predicates []expression.Expression) []e p.RightConditions = nil p.OtherConditions = nil nullSensitive := p.JoinType == AntiLeftOuterSemiJoin || p.JoinType == LeftOuterSemiJoin - joinConds, predicates = expression.PropConstOverOuterJoin(p.ctx, joinConds, predicates, outerTable.Schema(), innerTable.Schema(), nullSensitive) + joinConds, predicates = expression.PropConstOverOuterJoin(p.SCtx(), joinConds, predicates, outerTable.Schema(), innerTable.Schema(), nullSensitive) p.AttachOnConds(joinConds) return predicates } @@ -779,7 +779,7 @@ func (p *LogicalWindow) PredicatePushDown(predicates []expression.Expression, op // PredicatePushDown implements LogicalPlan PredicatePushDown interface. func (p *LogicalMemTable) PredicatePushDown(predicates []expression.Expression, _ *logicalOptimizeOp) ([]expression.Expression, LogicalPlan) { if p.Extractor != nil { - predicates = p.Extractor.Extract(p.ctx, p.schema, p.names, predicates) + predicates = p.Extractor.Extract(p.SCtx(), p.schema, p.names, predicates) } return predicates, p.self } @@ -1010,7 +1010,7 @@ func (p *LogicalCTE) PredicatePushDown(predicates []expression.Expression, _ *lo newPred = append(newPred, pushedPredicates[i].Clone()) ResolveExprAndReplace(newPred[i], p.cte.ColumnMap) } - p.cte.pushDownPredicates = append(p.cte.pushDownPredicates, expression.ComposeCNFCondition(p.ctx, newPred...)) + p.cte.pushDownPredicates = append(p.cte.pushDownPredicates, expression.ComposeCNFCondition(p.SCtx(), newPred...)) return predicates, p.self } diff --git a/planner/core/rule_predicate_simplification.go b/planner/core/rule_predicate_simplification.go index 403b212885fcc..230800ea754f0 100644 --- a/planner/core/rule_predicate_simplification.go +++ b/planner/core/rule_predicate_simplification.go @@ -156,8 +156,8 @@ func applyPredicateSimplification(sctx sessionctx.Context, predicates []expressi func (ds *DataSource) predicateSimplification(*logicalOptimizeOp) LogicalPlan { p := ds.self.(*DataSource) - p.pushedDownConds = applyPredicateSimplification(p.ctx, p.pushedDownConds) - p.allConds = applyPredicateSimplification(p.ctx, p.allConds) + p.pushedDownConds = applyPredicateSimplification(p.SCtx(), p.pushedDownConds) + p.allConds = applyPredicateSimplification(p.SCtx(), p.allConds) return p } diff --git a/planner/core/rule_topn_push_down.go b/planner/core/rule_topn_push_down.go index f8ccfb75bc117..3760d690284a0 100644 --- a/planner/core/rule_topn_push_down.go +++ b/planner/core/rule_topn_push_down.go @@ -70,7 +70,7 @@ func (lt *LogicalTopN) setChild(p LogicalPlan, opt *logicalOptimizeOp) LogicalPl Offset: lt.Offset, limitHints: lt.limitHints, PartitionBy: lt.GetPartitionBy(), - }.Init(lt.ctx, lt.blockOffset) + }.Init(lt.SCtx(), lt.SelectBlockOffset()) limit.SetChildren(p) appendTopNPushDownTraceStep(limit, p, opt) return limit @@ -94,7 +94,7 @@ func (ls *LogicalSort) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp) L } func (p *LogicalLimit) convertToTopN(opt *logicalOptimizeOp) *LogicalTopN { - topn := LogicalTopN{Offset: p.Offset, Count: p.Count, limitHints: p.limitHints}.Init(p.ctx, p.blockOffset) + topn := LogicalTopN{Offset: p.Offset, Count: p.Count, limitHints: p.limitHints}.Init(p.SCtx(), p.SelectBlockOffset()) appendConvertTopNTraceStep(p, topn, opt) return topn } @@ -111,7 +111,7 @@ func (p *LogicalUnionAll) pushDownTopN(topN *LogicalTopN, opt *logicalOptimizeOp for i, child := range p.children { var newTopN *LogicalTopN if topN != nil { - newTopN = LogicalTopN{Count: topN.Count + topN.Offset, limitHints: topN.limitHints}.Init(p.ctx, topN.blockOffset) + newTopN = LogicalTopN{Count: topN.Count + topN.Offset, limitHints: topN.limitHints}.Init(p.SCtx(), topN.SelectBlockOffset()) for _, by := range topN.ByItems { newTopN.ByItems = append(newTopN.ByItems, &util.ByItems{Expr: by.Expr, Desc: by.Desc}) } @@ -189,7 +189,7 @@ func (p *LogicalJoin) pushDownTopNToChild(topN *LogicalTopN, idx int, opt *logic Count: topN.Count + topN.Offset, ByItems: make([]*util.ByItems, len(topN.ByItems)), limitHints: topN.limitHints, - }.Init(topN.ctx, topN.blockOffset) + }.Init(topN.SCtx(), topN.SelectBlockOffset()) for i := range topN.ByItems { newTopN.ByItems[i] = topN.ByItems[i].Clone() } diff --git a/planner/core/runtime_filter.go b/planner/core/runtime_filter.go index ca209510ca97e..802a12ea77dd5 100644 --- a/planner/core/runtime_filter.go +++ b/planner/core/runtime_filter.go @@ -88,7 +88,7 @@ func NewRuntimeFilter(rfIDGenerator *util.IDGenerator, eqPredicate *expression.S targetExprUniqueID = eqPredicate.GetArgs()[1].(*expression.Column).UniqueID } - rfTypes := buildNode.ctx.GetSessionVars().GetRuntimeFilterTypes() + rfTypes := buildNode.SCtx().GetSessionVars().GetRuntimeFilterTypes() result := make([]*RuntimeFilter, 0, len(rfTypes)) for _, rfType := range rfTypes { rf := &RuntimeFilter{ @@ -143,12 +143,12 @@ func (rf *RuntimeFilter) String() string { var builder strings.Builder fmt.Fprintf(&builder, "id=%d", rf.id) builder.WriteString(", ") - fmt.Fprintf(&builder, "buildNodeID=%d", rf.buildNode.id) + fmt.Fprintf(&builder, "buildNodeID=%d", rf.buildNode.ID()) builder.WriteString(", ") if rf.targetNode == nil { fmt.Fprintf(&builder, "targetNodeID=nil") } else { - fmt.Fprintf(&builder, "targetNodeID=%d", rf.targetNode.id) + fmt.Fprintf(&builder, "targetNodeID=%d", rf.targetNode.ID()) } builder.WriteString(", ") fmt.Fprintf(&builder, "srcColumn=") @@ -181,12 +181,12 @@ func (rf *RuntimeFilter) Clone() *RuntimeFilter { if rf.buildNode == nil { cloned.buildNodeID = rf.buildNodeID } else { - cloned.buildNodeID = rf.buildNode.id + cloned.buildNodeID = rf.buildNode.ID() } if rf.targetNode == nil { cloned.targetNodeID = rf.targetNodeID } else { - cloned.targetNodeID = rf.targetNode.id + cloned.targetNodeID = rf.targetNode.ID() } for _, srcExpr := range rf.srcExprList { diff --git a/planner/core/runtime_filter_generator.go b/planner/core/runtime_filter_generator.go index b905f3dc1580a..38f1e591508a0 100644 --- a/planner/core/runtime_filter_generator.go +++ b/planner/core/runtime_filter_generator.go @@ -81,7 +81,7 @@ func (generator *RuntimeFilterGenerator) generateRuntimeFilterInterval(hashJoinP // precondition: the storage type of hash join must be TiFlash if hashJoinPlan.storeTp != kv.TiFlash { logutil.BgLogger().Warn("RF only support TiFlash compute engine while storage type of hash join node is not TiFlash", - zap.Int("PhysicalHashJoinId", hashJoinPlan.id), + zap.Int("PhysicalHashJoinId", hashJoinPlan.ID()), zap.String("StoreTP", hashJoinPlan.storeTp.Name())) return } @@ -114,17 +114,17 @@ func (generator *RuntimeFilterGenerator) assignRuntimeFilter(physicalTableScan * for _, runtimeFilter := range currentColumnRFList { // compute rf mode var rfMode RuntimeFilterMode - if cacheBuildNodeIDToRFMode[runtimeFilter.buildNode.id] != 0 { - rfMode = cacheBuildNodeIDToRFMode[runtimeFilter.buildNode.id] + if cacheBuildNodeIDToRFMode[runtimeFilter.buildNode.ID()] != 0 { + rfMode = cacheBuildNodeIDToRFMode[runtimeFilter.buildNode.ID()] } else { rfMode = generator.calculateRFMode(runtimeFilter.buildNode, physicalTableScan) - cacheBuildNodeIDToRFMode[runtimeFilter.buildNode.id] = rfMode + cacheBuildNodeIDToRFMode[runtimeFilter.buildNode.ID()] = rfMode } // todo support global RF if rfMode == variable.RFGlobal { logutil.BgLogger().Debug("Now we don't support global RF. Remove it", - zap.Int("BuildNodeId", runtimeFilter.buildNode.id), - zap.Int("TargetNodeId", physicalTableScan.id)) + zap.Int("BuildNodeId", runtimeFilter.buildNode.ID()), + zap.Int("TargetNodeId", physicalTableScan.ID())) continue } runtimeFilter.rfMode = rfMode @@ -162,7 +162,7 @@ func (*RuntimeFilterGenerator) matchRFJoinType(hashJoinPlan *PhysicalHashJoin) b if hashJoinPlan.JoinType == LeftOuterJoin || hashJoinPlan.JoinType == AntiSemiJoin || hashJoinPlan.JoinType == LeftOuterSemiJoin || hashJoinPlan.JoinType == AntiLeftOuterSemiJoin { logutil.BgLogger().Debug("Join type does not match RF pattern when build side is on the right", - zap.Int32("PlanNodeId", int32(hashJoinPlan.id)), + zap.Int32("PlanNodeId", int32(hashJoinPlan.ID())), zap.String("JoinType", hashJoinPlan.JoinType.String())) return false } @@ -170,7 +170,7 @@ func (*RuntimeFilterGenerator) matchRFJoinType(hashJoinPlan *PhysicalHashJoin) b // case2: build side is on the left if hashJoinPlan.JoinType == RightOuterJoin { logutil.BgLogger().Debug("Join type does not match RF pattern when build side is on the left", - zap.Int32("PlanNodeId", int32(hashJoinPlan.id)), + zap.Int32("PlanNodeId", int32(hashJoinPlan.ID())), zap.String("JoinType", hashJoinPlan.JoinType.String())) return false } @@ -233,7 +233,7 @@ func (generator *RuntimeFilterGenerator) belongsToSameFragment(currentNode Physi // terminal traversal return false case *PhysicalTableScan: - if currentNode.ID() == targetNode.id { + if currentNode.ID() == targetNode.ID() { return true } return false diff --git a/planner/core/scalar_subq_expression.go b/planner/core/scalar_subq_expression.go index 9d9f8c478eaef..922309077422b 100644 --- a/planner/core/scalar_subq_expression.go +++ b/planner/core/scalar_subq_expression.go @@ -22,6 +22,7 @@ import ( "github.com/pingcap/errors" "github.com/pingcap/tidb/expression" "github.com/pingcap/tidb/infoschema" + "github.com/pingcap/tidb/planner/core/internal/base" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/sessionctx/stmtctx" "github.com/pingcap/tidb/types" @@ -31,7 +32,7 @@ import ( // ScalarSubqueryEvalCtx store the plan for the subquery, used by ScalarSubQueryExpr. type ScalarSubqueryEvalCtx struct { - basePlan + base.Plan // The context for evaluating the subquery. scalarSubQuery PhysicalPlan diff --git a/planner/core/stats.go b/planner/core/stats.go index 42b14cfa95e9d..14ab62502bd51 100644 --- a/planner/core/stats.go +++ b/planner/core/stats.go @@ -42,13 +42,13 @@ import ( ) func (p *basePhysicalPlan) StatsCount() float64 { - return p.stats.RowCount + return p.StatsInfo().RowCount } // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalTableDual) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } profile := &property.StatsInfo{ RowCount: float64(p.RowCount), @@ -57,14 +57,14 @@ func (p *LogicalTableDual) DeriveStats(_ []*property.StatsInfo, selfSchema *expr for _, col := range selfSchema.Columns { profile.ColNDVs[col.UniqueID] = float64(p.RowCount) } - p.stats = profile - return p.stats, nil + p.SetStats(profile) + return p.StatsInfo(), nil } // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalMemTable) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } statsTable := statistics.PseudoTable(p.TableInfo) stats := &property.StatsInfo{ @@ -76,18 +76,18 @@ func (p *LogicalMemTable) DeriveStats(_ []*property.StatsInfo, selfSchema *expre for _, col := range selfSchema.Columns { stats.ColNDVs[col.UniqueID] = float64(statsTable.RealtimeCount) } - p.stats = stats - return p.stats, nil + p.SetStats(stats) + return p.StatsInfo(), nil } // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalShow) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } // A fake count, just to avoid panic now. - p.stats = getFakeStats(selfSchema) - return p.stats, nil + p.SetStats(getFakeStats(selfSchema)) + return p.StatsInfo(), nil } func getFakeStats(schema *expression.Schema) *property.StatsInfo { @@ -103,12 +103,12 @@ func getFakeStats(schema *expression.Schema) *property.StatsInfo { // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalShowDDLJobs) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } // A fake count, just to avoid panic now. - p.stats = getFakeStats(selfSchema) - return p.stats, nil + p.SetStats(getFakeStats(selfSchema)) + return p.StatsInfo(), nil } // RecursiveDeriveStats4Test is a exporter just for test. @@ -118,7 +118,7 @@ func RecursiveDeriveStats4Test(p LogicalPlan) (*property.StatsInfo, error) { // GetStats4Test is a exporter just for test. func GetStats4Test(p LogicalPlan) *property.StatsInfo { - return p.statsInfo() + return p.StatsInfo() } func (p *baseLogicalPlan) recursiveDeriveStats(colGroups [][]*expression.Column) (*property.StatsInfo, error) { @@ -144,15 +144,15 @@ func (*baseLogicalPlan) ExtractColGroups(_ [][]*expression.Column) [][]*expressi // DeriveStats implement LogicalPlan DeriveStats interface. func (p *baseLogicalPlan) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { if len(childStats) == 1 { - p.stats = childStats[0] - return p.stats, nil + p.SetStats(childStats[0]) + return p.StatsInfo(), nil } if len(childStats) > 1 { err := ErrInternal.GenWithStack("LogicalPlans with more than one child should implement their own DeriveStats().") return nil, err } - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } profile := &property.StatsInfo{ RowCount: float64(1), @@ -161,7 +161,7 @@ func (p *baseLogicalPlan) DeriveStats(childStats []*property.StatsInfo, selfSche for _, col := range selfSchema.Columns { profile.ColNDVs[col.UniqueID] = 1 } - p.stats = profile + p.SetStats(profile) return profile, nil } @@ -284,7 +284,7 @@ func (ds *DataSource) initStats(colGroups [][]*expression.Column) { return } if ds.statisticTable == nil { - ds.statisticTable = getStatsTable(ds.ctx, ds.tableInfo, ds.physicalTableID) + ds.statisticTable = getStatsTable(ds.SCtx(), ds.tableInfo, ds.physicalTableID) } tableStats := &property.StatsInfo{ RowCount: float64(ds.statisticTable.RealtimeCount), @@ -296,8 +296,8 @@ func (ds *DataSource) initStats(colGroups [][]*expression.Column) { tableStats.StatsVersion = statistics.PseudoVersion } - statsRecord := ds.ctx.GetSessionVars().StmtCtx.GetUsedStatsInfo(true) - name, tblInfo := getTblInfoForUsedStatsByPhysicalID(ds.ctx, ds.physicalTableID) + statsRecord := ds.SCtx().GetSessionVars().StmtCtx.GetUsedStatsInfo(true) + name, tblInfo := getTblInfoForUsedStatsByPhysicalID(ds.SCtx(), ds.physicalTableID) statsRecord[ds.physicalTableID] = &stmtctx.UsedStatsInfoForTable{ Name: name, TblInfo: tblInfo, @@ -315,18 +315,18 @@ func (ds *DataSource) initStats(colGroups [][]*expression.Column) { } func (ds *DataSource) deriveStatsByFilter(conds expression.CNFExprs, filledPaths []*util.AccessPath) *property.StatsInfo { - if ds.ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugtrace.EnterContextCommon(ds.ctx) - defer debugtrace.LeaveContextCommon(ds.ctx) + if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(ds.SCtx()) + defer debugtrace.LeaveContextCommon(ds.SCtx()) } - selectivity, nodes, err := ds.tableStats.HistColl.Selectivity(ds.ctx, conds, filledPaths) + selectivity, nodes, err := ds.tableStats.HistColl.Selectivity(ds.SCtx(), conds, filledPaths) if err != nil { logutil.BgLogger().Debug("something wrong happened, use the default selectivity", zap.Error(err)) selectivity = SelectionFactor } stats := ds.tableStats.Scale(selectivity) - if ds.ctx.GetSessionVars().OptimizerSelectivityLevel >= 1 { - stats.HistColl = stats.HistColl.NewHistCollBySelectivity(ds.ctx, nodes) + if ds.SCtx().GetSessionVars().OptimizerSelectivityLevel >= 1 { + stats.HistColl = stats.HistColl.NewHistCollBySelectivity(ds.SCtx(), nodes) } return stats } @@ -334,9 +334,9 @@ func (ds *DataSource) deriveStatsByFilter(conds expression.CNFExprs, filledPaths // We bind logic of derivePathStats and tryHeuristics together. When some path matches the heuristic rule, we don't need // to derive stats of subsequent paths. In this way we can save unnecessary computation of derivePathStats. func (ds *DataSource) derivePathStatsAndTryHeuristics() error { - if ds.ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugtrace.EnterContextCommon(ds.ctx) - defer debugtrace.LeaveContextCommon(ds.ctx) + if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(ds.SCtx()) + defer debugtrace.LeaveContextCommon(ds.SCtx()) } uniqueIdxsWithDoubleScan := make([]*util.AccessPath, 0, len(ds.possibleAccessPaths)) singleScanIdxs := make([]*util.AccessPath, 0, len(ds.possibleAccessPaths)) @@ -445,10 +445,10 @@ func (ds *DataSource) derivePathStatsAndTryHeuristics() error { sb.WriteString(" with double scan") } } - if ds.ctx.GetSessionVars().StmtCtx.InVerboseExplain { - ds.ctx.GetSessionVars().StmtCtx.AppendNote(errors.New(sb.String())) + if ds.SCtx().GetSessionVars().StmtCtx.InVerboseExplain { + ds.SCtx().GetSessionVars().StmtCtx.AppendNote(errors.New(sb.String())) } else { - ds.ctx.GetSessionVars().StmtCtx.AppendExtraNote(errors.New(sb.String())) + ds.SCtx().GetSessionVars().StmtCtx.AppendExtraNote(errors.New(sb.String())) } } return nil @@ -456,23 +456,23 @@ func (ds *DataSource) derivePathStatsAndTryHeuristics() error { // DeriveStats implement LogicalPlan DeriveStats interface. func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error) { - if ds.stats != nil && len(colGroups) == 0 { - return ds.stats, nil + if ds.StatsInfo() != nil && len(colGroups) == 0 { + return ds.StatsInfo(), nil } ds.initStats(colGroups) - if ds.stats != nil { + if ds.StatsInfo() != nil { // Just reload the GroupNDVs. - selectivity := ds.stats.RowCount / ds.tableStats.RowCount - ds.stats = ds.tableStats.Scale(selectivity) - return ds.stats, nil + selectivity := ds.StatsInfo().RowCount / ds.tableStats.RowCount + ds.SetStats(ds.tableStats.Scale(selectivity)) + return ds.StatsInfo(), nil } - if ds.ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugtrace.EnterContextCommon(ds.ctx) - defer debugtrace.LeaveContextCommon(ds.ctx) + if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugtrace.EnterContextCommon(ds.SCtx()) + defer debugtrace.LeaveContextCommon(ds.SCtx()) } // PushDownNot here can convert query 'not (a != 1)' to 'a = 1'. for i, expr := range ds.pushedDownConds { - ds.pushedDownConds[i] = expression.PushDownNot(ds.ctx, expr) + ds.pushedDownConds[i] = expression.PushDownNot(ds.SCtx(), expr) } for _, path := range ds.possibleAccessPaths { if path.IsTablePath() { @@ -485,7 +485,7 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, } // TODO: Can we move ds.deriveStatsByFilter after pruning by heuristics? In this way some computation can be avoided // when ds.possibleAccessPaths are pruned. - ds.stats = ds.deriveStatsByFilter(ds.pushedDownConds, ds.possibleAccessPaths) + ds.SetStats(ds.deriveStatsByFilter(ds.pushedDownConds, ds.possibleAccessPaths)) err := ds.derivePathStatsAndTryHeuristics() if err != nil { return nil, err @@ -495,12 +495,12 @@ func (ds *DataSource) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, return nil, err } - if ds.ctx.GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { - debugTraceAccessPaths(ds.ctx, ds.possibleAccessPaths) + if ds.SCtx().GetSessionVars().StmtCtx.EnableOptimizerDebugTrace { + debugTraceAccessPaths(ds.SCtx(), ds.possibleAccessPaths) } ds.accessPathMinSelectivity = getMinSelectivityFromPaths(ds.possibleAccessPaths, float64(ds.TblColHists.RealtimeCount)) - return ds.stats, nil + return ds.StatsInfo(), nil } func getMinSelectivityFromPaths(paths []*util.AccessPath, totalRowCount float64) float64 { @@ -527,14 +527,14 @@ func (ts *LogicalTableScan) DeriveStats(_ []*property.StatsInfo, _ *expression.S for i, expr := range ts.AccessConds { // TODO The expressions may be shared by TableScan and several IndexScans, there would be redundant // `PushDownNot` function call in multiple `DeriveStats` then. - ts.AccessConds[i] = expression.PushDownNot(ts.ctx, expr) + ts.AccessConds[i] = expression.PushDownNot(ts.SCtx(), expr) } - ts.stats = ts.Source.deriveStatsByFilter(ts.AccessConds, nil) + ts.SetStats(ts.Source.deriveStatsByFilter(ts.AccessConds, nil)) // ts.Handle could be nil if PK is Handle, and PK column has been pruned. // TODO: support clustered index. if ts.HandleCols != nil { // TODO: restrict mem usage of table ranges. - ts.Ranges, _, _, err = ranger.BuildTableRange(ts.AccessConds, ts.ctx, ts.HandleCols.GetCol(0).RetType, 0) + ts.Ranges, _, _, err = ranger.BuildTableRange(ts.AccessConds, ts.SCtx(), ts.HandleCols.GetCol(0).RetType, 0) } else { isUnsigned := false if ts.Source.tableInfo.PKIsHandle { @@ -547,16 +547,16 @@ func (ts *LogicalTableScan) DeriveStats(_ []*property.StatsInfo, _ *expression.S if err != nil { return nil, err } - return ts.stats, nil + return ts.StatsInfo(), nil } // DeriveStats implements LogicalPlan DeriveStats interface. func (is *LogicalIndexScan) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { is.Source.initStats(nil) for i, expr := range is.AccessConds { - is.AccessConds[i] = expression.PushDownNot(is.ctx, expr) + is.AccessConds[i] = expression.PushDownNot(is.SCtx(), expr) } - is.stats = is.Source.deriveStatsByFilter(is.AccessConds, nil) + is.SetStats(is.Source.deriveStatsByFilter(is.AccessConds, nil)) if len(is.AccessConds) == 0 { is.Ranges = ranger.FullRange() } @@ -569,34 +569,34 @@ func (is *LogicalIndexScan) DeriveStats(_ []*property.StatsInfo, selfSchema *exp is.IdxColLens = append(is.IdxColLens, types.UnspecifiedLength) } } - return is.stats, nil + return is.StatsInfo(), nil } // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalSelection) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } - p.stats = childStats[0].Scale(SelectionFactor) - p.stats.GroupNDVs = nil - return p.stats, nil + p.SetStats(childStats[0].Scale(SelectionFactor)) + p.StatsInfo().GroupNDVs = nil + return p.StatsInfo(), nil } // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalUnionAll) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } - p.stats = &property.StatsInfo{ + p.SetStats(&property.StatsInfo{ ColNDVs: make(map[int64]float64, selfSchema.Len()), - } + }) for _, childProfile := range childStats { - p.stats.RowCount += childProfile.RowCount + p.StatsInfo().RowCount += childProfile.RowCount for _, col := range selfSchema.Columns { - p.stats.ColNDVs[col.UniqueID] += childProfile.ColNDVs[col.UniqueID] + p.StatsInfo().ColNDVs[col.UniqueID] += childProfile.ColNDVs[col.UniqueID] } } - return p.stats, nil + return p.StatsInfo(), nil } func deriveLimitStats(childProfile *property.StatsInfo, limitCount float64) *property.StatsInfo { @@ -612,20 +612,20 @@ func deriveLimitStats(childProfile *property.StatsInfo, limitCount float64) *pro // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalLimit) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } - p.stats = deriveLimitStats(childStats[0], float64(p.Count)) - return p.stats, nil + p.SetStats(deriveLimitStats(childStats[0], float64(p.Count))) + return p.StatsInfo(), nil } // DeriveStats implement LogicalPlan DeriveStats interface. func (lt *LogicalTopN) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if lt.stats != nil { - return lt.stats, nil + if lt.StatsInfo() != nil { + return lt.StatsInfo(), nil } - lt.stats = deriveLimitStats(childStats[0], float64(lt.Count)) - return lt.stats, nil + lt.SetStats(deriveLimitStats(childStats[0], float64(lt.Count))) + return lt.StatsInfo(), nil } func getGroupNDV4Cols(cols []*expression.Column, stats *property.StatsInfo) *property.GroupNDV { @@ -721,21 +721,21 @@ func (p *LogicalProjection) getGroupNDVs(colGroups [][]*expression.Column, child // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalProjection) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, childSchema []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error) { childProfile := childStats[0] - if p.stats != nil { + if p.StatsInfo() != nil { // Reload GroupNDVs since colGroups may have changed. - p.stats.GroupNDVs = p.getGroupNDVs(colGroups, childProfile, selfSchema) - return p.stats, nil + p.StatsInfo().GroupNDVs = p.getGroupNDVs(colGroups, childProfile, selfSchema) + return p.StatsInfo(), nil } - p.stats = &property.StatsInfo{ + p.SetStats(&property.StatsInfo{ RowCount: childProfile.RowCount, ColNDVs: make(map[int64]float64, len(p.Exprs)), - } + }) for i, expr := range p.Exprs { cols := expression.ExtractColumns(expr) - p.stats.ColNDVs[selfSchema.Columns[i].UniqueID], _ = getColsNDVWithMatchedLen(cols, childSchema[0], childProfile) + p.StatsInfo().ColNDVs[selfSchema.Columns[i].UniqueID], _ = getColsNDVWithMatchedLen(cols, childSchema[0], childProfile) } - p.stats.GroupNDVs = p.getGroupNDVs(colGroups, childProfile, selfSchema) - return p.stats, nil + p.StatsInfo().GroupNDVs = p.getGroupNDVs(colGroups, childProfile, selfSchema) + return p.StatsInfo(), nil } // ExtractColGroups implements LogicalPlan ExtractColGroups interface. @@ -790,23 +790,23 @@ func (la *LogicalAggregation) DeriveStats(childStats []*property.StatsInfo, self cols := expression.ExtractColumns(gbyExpr) gbyCols = append(gbyCols, cols...) } - if la.stats != nil { + if la.StatsInfo() != nil { // Reload GroupNDVs since colGroups may have changed. - la.stats.GroupNDVs = la.getGroupNDVs(colGroups, childProfile, gbyCols) - return la.stats, nil + la.StatsInfo().GroupNDVs = la.getGroupNDVs(colGroups, childProfile, gbyCols) + return la.StatsInfo(), nil } ndv, _ := getColsNDVWithMatchedLen(gbyCols, childSchema[0], childProfile) - la.stats = &property.StatsInfo{ + la.SetStats(&property.StatsInfo{ RowCount: ndv, ColNDVs: make(map[int64]float64, selfSchema.Len()), - } + }) // We cannot estimate the ColNDVs for every output, so we use a conservative strategy. for _, col := range selfSchema.Columns { - la.stats.ColNDVs[col.UniqueID] = ndv + la.StatsInfo().ColNDVs[col.UniqueID] = ndv } la.inputCount = childProfile.RowCount - la.stats.GroupNDVs = la.getGroupNDVs(colGroups, childProfile, gbyCols) - return la.stats, nil + la.StatsInfo().GroupNDVs = la.getGroupNDVs(colGroups, childProfile, gbyCols) + return la.StatsInfo(), nil } // ExtractColGroups implements LogicalPlan ExtractColGroups interface. @@ -848,10 +848,10 @@ func (p *LogicalJoin) getGroupNDVs(colGroups [][]*expression.Column, childStats // 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 (p *LogicalJoin) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, childSchema []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { + if p.StatsInfo() != nil { // Reload GroupNDVs since colGroups may have changed. - p.stats.GroupNDVs = p.getGroupNDVs(colGroups, childStats) - return p.stats, nil + p.StatsInfo().GroupNDVs = p.getGroupNDVs(colGroups, childStats) + return p.StatsInfo(), nil } leftProfile, rightProfile := childStats[0], childStats[1] leftJoinKeys, rightJoinKeys, _, _ := p.GetJoinKeys() @@ -867,26 +867,26 @@ func (p *LogicalJoin) DeriveStats(childStats []*property.StatsInfo, selfSchema * } p.equalCondOutCnt = helper.estimate() if p.JoinType == SemiJoin || p.JoinType == AntiSemiJoin { - p.stats = &property.StatsInfo{ + p.SetStats(&property.StatsInfo{ RowCount: leftProfile.RowCount * SelectionFactor, ColNDVs: make(map[int64]float64, len(leftProfile.ColNDVs)), - } + }) for id, c := range leftProfile.ColNDVs { - p.stats.ColNDVs[id] = c * SelectionFactor + p.StatsInfo().ColNDVs[id] = c * SelectionFactor } - return p.stats, nil + return p.StatsInfo(), nil } if p.JoinType == LeftOuterSemiJoin || p.JoinType == AntiLeftOuterSemiJoin { - p.stats = &property.StatsInfo{ + p.SetStats(&property.StatsInfo{ RowCount: leftProfile.RowCount, ColNDVs: make(map[int64]float64, selfSchema.Len()), - } + }) for id, c := range leftProfile.ColNDVs { - p.stats.ColNDVs[id] = c + p.StatsInfo().ColNDVs[id] = c } - p.stats.ColNDVs[selfSchema.Columns[selfSchema.Len()-1].UniqueID] = 2.0 - p.stats.GroupNDVs = p.getGroupNDVs(colGroups, childStats) - return p.stats, nil + p.StatsInfo().ColNDVs[selfSchema.Columns[selfSchema.Len()-1].UniqueID] = 2.0 + p.StatsInfo().GroupNDVs = p.getGroupNDVs(colGroups, childStats) + return p.StatsInfo(), nil } count := p.equalCondOutCnt if p.JoinType == LeftOuterJoin { @@ -901,12 +901,12 @@ func (p *LogicalJoin) DeriveStats(childStats []*property.StatsInfo, selfSchema * for id, c := range rightProfile.ColNDVs { colNDVs[id] = math.Min(c, count) } - p.stats = &property.StatsInfo{ + p.SetStats(&property.StatsInfo{ RowCount: count, ColNDVs: colNDVs, - } - p.stats.GroupNDVs = p.getGroupNDVs(colGroups, childStats) - return p.stats, nil + }) + p.StatsInfo().GroupNDVs = p.getGroupNDVs(colGroups, childStats) + return p.StatsInfo(), nil } // ExtractColGroups implements LogicalPlan ExtractColGroups interface. @@ -980,28 +980,28 @@ func (la *LogicalApply) getGroupNDVs(colGroups [][]*expression.Column, childStat // DeriveStats implement LogicalPlan DeriveStats interface. func (la *LogicalApply) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, childSchema []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error) { - if la.stats != nil { + if la.StatsInfo() != nil { // Reload GroupNDVs since colGroups may have changed. - la.stats.GroupNDVs = la.getGroupNDVs(colGroups, childStats) - return la.stats, nil + la.StatsInfo().GroupNDVs = la.getGroupNDVs(colGroups, childStats) + return la.StatsInfo(), nil } leftProfile := childStats[0] - la.stats = &property.StatsInfo{ + la.SetStats(&property.StatsInfo{ RowCount: leftProfile.RowCount, ColNDVs: make(map[int64]float64, selfSchema.Len()), - } + }) for id, c := range leftProfile.ColNDVs { - la.stats.ColNDVs[id] = c + la.StatsInfo().ColNDVs[id] = c } if la.JoinType == LeftOuterSemiJoin || la.JoinType == AntiLeftOuterSemiJoin { - la.stats.ColNDVs[selfSchema.Columns[selfSchema.Len()-1].UniqueID] = 2.0 + la.StatsInfo().ColNDVs[selfSchema.Columns[selfSchema.Len()-1].UniqueID] = 2.0 } else { for i := childSchema[0].Len(); i < selfSchema.Len(); i++ { - la.stats.ColNDVs[selfSchema.Columns[i].UniqueID] = leftProfile.RowCount + la.StatsInfo().ColNDVs[selfSchema.Columns[i].UniqueID] = leftProfile.RowCount } } - la.stats.GroupNDVs = la.getGroupNDVs(colGroups, childStats) - return la.stats, nil + la.StatsInfo().GroupNDVs = la.getGroupNDVs(colGroups, childStats) + return la.StatsInfo(), nil } // ExtractColGroups implements LogicalPlan ExtractColGroups interface. @@ -1039,11 +1039,11 @@ func getSingletonStats(schema *expression.Schema) *property.StatsInfo { // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalMaxOneRow) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } - p.stats = getSingletonStats(selfSchema) - return p.stats, nil + p.SetStats(getSingletonStats(selfSchema)) + return p.StatsInfo(), nil } func (*LogicalWindow) getGroupNDVs(colGroups [][]*expression.Column, childStats []*property.StatsInfo) []property.GroupNDV { @@ -1055,26 +1055,26 @@ func (*LogicalWindow) getGroupNDVs(colGroups [][]*expression.Column, childStats // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalWindow) DeriveStats(childStats []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, colGroups [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { + if p.StatsInfo() != nil { // Reload GroupNDVs since colGroups may have changed. - p.stats.GroupNDVs = p.getGroupNDVs(colGroups, childStats) - return p.stats, nil + p.StatsInfo().GroupNDVs = p.getGroupNDVs(colGroups, childStats) + return p.StatsInfo(), nil } childProfile := childStats[0] - p.stats = &property.StatsInfo{ + p.SetStats(&property.StatsInfo{ RowCount: childProfile.RowCount, ColNDVs: make(map[int64]float64, selfSchema.Len()), - } + }) childLen := selfSchema.Len() - len(p.WindowFuncDescs) for i := 0; i < childLen; i++ { id := selfSchema.Columns[i].UniqueID - p.stats.ColNDVs[id] = childProfile.ColNDVs[id] + p.StatsInfo().ColNDVs[id] = childProfile.ColNDVs[id] } for i := childLen; i < selfSchema.Len(); i++ { - p.stats.ColNDVs[selfSchema.Columns[i].UniqueID] = childProfile.RowCount + p.StatsInfo().ColNDVs[selfSchema.Columns[i].UniqueID] = childProfile.RowCount } - p.stats.GroupNDVs = p.getGroupNDVs(colGroups, childStats) - return p.stats, nil + p.StatsInfo().GroupNDVs = p.getGroupNDVs(colGroups, childStats) + return p.StatsInfo(), nil } // ExtractColGroups implements LogicalPlan ExtractColGroups interface. @@ -1096,21 +1096,21 @@ func (p *LogicalWindow) ExtractColGroups(colGroups [][]*expression.Column) [][]* // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalCTE) DeriveStats(_ []*property.StatsInfo, selfSchema *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } var err error if p.cte.seedPartPhysicalPlan == nil { // Build push-downed predicates. if len(p.cte.pushDownPredicates) > 0 { - newCond := expression.ComposeDNFCondition(p.ctx, p.cte.pushDownPredicates...) + newCond := expression.ComposeDNFCondition(p.SCtx(), p.cte.pushDownPredicates...) newSel := LogicalSelection{Conditions: []expression.Expression{newCond}}.Init(p.SCtx(), p.cte.seedPartLogicalPlan.SelectBlockOffset()) newSel.SetChildren(p.cte.seedPartLogicalPlan) p.cte.seedPartLogicalPlan = newSel p.cte.optFlag |= flagPredicatePushDown } - p.cte.seedPartLogicalPlan, p.cte.seedPartPhysicalPlan, _, err = DoOptimizeAndLogicAsRet(context.TODO(), p.ctx, p.cte.optFlag, p.cte.seedPartLogicalPlan) + p.cte.seedPartLogicalPlan, p.cte.seedPartPhysicalPlan, _, err = DoOptimizeAndLogicAsRet(context.TODO(), p.SCtx(), p.cte.optFlag, p.cte.seedPartLogicalPlan) if err != nil { return nil, err } @@ -1118,47 +1118,47 @@ func (p *LogicalCTE) DeriveStats(_ []*property.StatsInfo, selfSchema *expression if p.onlyUsedAsStorage { p.SetChildren(p.cte.seedPartLogicalPlan) } - resStat := p.cte.seedPartPhysicalPlan.Stats() + resStat := p.cte.seedPartPhysicalPlan.StatsInfo() // Changing the pointer so that seedStat in LogicalCTETable can get the new stat. *p.seedStat = *resStat - p.stats = &property.StatsInfo{ + p.SetStats(&property.StatsInfo{ RowCount: resStat.RowCount, ColNDVs: make(map[int64]float64, selfSchema.Len()), - } + }) for i, col := range selfSchema.Columns { - p.stats.ColNDVs[col.UniqueID] += resStat.ColNDVs[p.cte.seedPartLogicalPlan.Schema().Columns[i].UniqueID] + p.StatsInfo().ColNDVs[col.UniqueID] += resStat.ColNDVs[p.cte.seedPartLogicalPlan.Schema().Columns[i].UniqueID] } if p.cte.recursivePartLogicalPlan != nil { if p.cte.recursivePartPhysicalPlan == nil { - p.cte.recursivePartPhysicalPlan, _, err = DoOptimize(context.TODO(), p.ctx, p.cte.optFlag, p.cte.recursivePartLogicalPlan) + p.cte.recursivePartPhysicalPlan, _, err = DoOptimize(context.TODO(), p.SCtx(), p.cte.optFlag, p.cte.recursivePartLogicalPlan) if err != nil { return nil, err } } - recurStat := p.cte.recursivePartPhysicalPlan.Stats() + recurStat := p.cte.recursivePartPhysicalPlan.StatsInfo() for i, col := range selfSchema.Columns { - p.stats.ColNDVs[col.UniqueID] += recurStat.ColNDVs[p.cte.recursivePartLogicalPlan.Schema().Columns[i].UniqueID] + p.StatsInfo().ColNDVs[col.UniqueID] += recurStat.ColNDVs[p.cte.recursivePartLogicalPlan.Schema().Columns[i].UniqueID] } if p.cte.IsDistinct { - p.stats.RowCount, _ = getColsNDVWithMatchedLen(p.schema.Columns, p.schema, p.stats) + p.StatsInfo().RowCount, _ = getColsNDVWithMatchedLen(p.schema.Columns, p.schema, p.StatsInfo()) } else { - p.stats.RowCount += recurStat.RowCount + p.StatsInfo().RowCount += recurStat.RowCount } } - return p.stats, nil + return p.StatsInfo(), nil } // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalCTETable) DeriveStats(_ []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - if p.stats != nil { - return p.stats, nil + if p.StatsInfo() != nil { + return p.StatsInfo(), nil } - p.stats = p.seedStat - return p.stats, nil + p.SetStats(p.seedStat) + return p.StatsInfo(), nil } // DeriveStats implement LogicalPlan DeriveStats interface. func (p *LogicalSequence) DeriveStats(childStats []*property.StatsInfo, _ *expression.Schema, _ []*expression.Schema, _ [][]*expression.Column) (*property.StatsInfo, error) { - p.stats = childStats[len(childStats)-1] - return p.stats, nil + p.SetStats(childStats[len(childStats)-1]) + return p.StatsInfo(), nil } diff --git a/planner/core/task.go b/planner/core/task.go index 1eb58da4f0b12..174b3463f846b 100644 --- a/planner/core/task.go +++ b/planner/core/task.go @@ -26,6 +26,7 @@ import ( "github.com/pingcap/tidb/parser/charset" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/core/internal/base" "github.com/pingcap/tidb/planner/property" "github.com/pingcap/tidb/planner/util" "github.com/pingcap/tidb/sessionctx" @@ -110,9 +111,9 @@ func (t *rootTask) invalid() bool { func (t *copTask) count() float64 { if t.indexPlanFinished { - return t.tablePlan.statsInfo().RowCount + return t.tablePlan.StatsInfo().RowCount } - return t.indexPlan.statsInfo().RowCount + return t.indexPlan.StatsInfo().RowCount } func (t *copTask) copy() task { @@ -157,11 +158,11 @@ func (t *copTask) finishIndexPlan() { // We need a elegant way to solve the stats of index merge in this case. if t.tablePlan != nil && t.indexPlan != nil { ts := t.tablePlan.(*PhysicalTableScan) - originStats := ts.stats - ts.stats = t.indexPlan.statsInfo() + originStats := ts.StatsInfo() + ts.SetStats(t.indexPlan.StatsInfo()) if originStats != nil { // keep the original stats version - ts.stats.StatsVersion = originStats.StatsVersion + ts.StatsInfo().StatsVersion = originStats.StatsVersion } } } @@ -220,7 +221,7 @@ func (t *copTask) MemoryUsage() (sum int64) { } func (p *basePhysicalPlan) attach2Task(tasks ...task) task { - t := tasks[0].convertToRootTask(p.ctx) + t := tasks[0].convertToRootTask(p.SCtx()) return attachPlan2Task(p.self, t) } @@ -232,7 +233,7 @@ func (p *PhysicalUnionScan) attach2Task(tasks ...task) task { // Convert unionScan->selection->projection to projection->unionScan->selection. sel.SetChildren(pj.children...) p.SetChildren(sel) - p.stats = tasks[0].plan().statsInfo() + p.SetStats(tasks[0].plan().StatsInfo()) rt, _ := tasks[0].(*rootTask) rt.p = p pj.SetChildren(p) @@ -242,19 +243,19 @@ func (p *PhysicalUnionScan) attach2Task(tasks ...task) task { if pj, ok := tasks[0].plan().(*PhysicalProjection); ok { // Convert unionScan->projection to projection->unionScan, because unionScan can't handle projection as its children. p.SetChildren(pj.children...) - p.stats = tasks[0].plan().statsInfo() + p.SetStats(tasks[0].plan().StatsInfo()) rt, _ := tasks[0].(*rootTask) rt.p = pj.children[0] pj.SetChildren(p) return pj.attach2Task(p.basePhysicalPlan.attach2Task(tasks...)) } - p.stats = tasks[0].plan().statsInfo() + p.SetStats(tasks[0].plan().StatsInfo()) return p.basePhysicalPlan.attach2Task(tasks...) } func (p *PhysicalApply) attach2Task(tasks ...task) task { - lTask := tasks[0].convertToRootTask(p.ctx) - rTask := tasks[1].convertToRootTask(p.ctx) + lTask := tasks[0].convertToRootTask(p.SCtx()) + rTask := tasks[1].convertToRootTask(p.SCtx()) p.SetChildren(lTask.plan(), rTask.plan()) p.schema = BuildPhysicalJoinSchema(p.JoinType, p) t := &rootTask{ @@ -265,7 +266,7 @@ func (p *PhysicalApply) attach2Task(tasks ...task) task { func (p *PhysicalIndexMergeJoin) attach2Task(tasks ...task) task { innerTask := p.innerTask - outerTask := tasks[1-p.InnerChildIdx].convertToRootTask(p.ctx) + outerTask := tasks[1-p.InnerChildIdx].convertToRootTask(p.SCtx()) if p.InnerChildIdx == 1 { p.SetChildren(outerTask.plan(), innerTask.plan()) } else { @@ -279,7 +280,7 @@ func (p *PhysicalIndexMergeJoin) attach2Task(tasks ...task) task { func (p *PhysicalIndexHashJoin) attach2Task(tasks ...task) task { innerTask := p.innerTask - outerTask := tasks[1-p.InnerChildIdx].convertToRootTask(p.ctx) + outerTask := tasks[1-p.InnerChildIdx].convertToRootTask(p.SCtx()) if p.InnerChildIdx == 1 { p.SetChildren(outerTask.plan(), innerTask.plan()) } else { @@ -293,7 +294,7 @@ func (p *PhysicalIndexHashJoin) attach2Task(tasks ...task) task { func (p *PhysicalIndexJoin) attach2Task(tasks ...task) task { innerTask := p.innerTask - outerTask := tasks[1-p.InnerChildIdx].convertToRootTask(p.ctx) + outerTask := tasks[1-p.InnerChildIdx].convertToRootTask(p.SCtx()) if p.InnerChildIdx == 1 { p.SetChildren(outerTask.plan(), innerTask.plan()) } else { @@ -321,8 +322,8 @@ func (p *PhysicalHashJoin) attach2Task(tasks ...task) task { if p.storeTp == kv.TiFlash { return p.attach2TaskForTiFlash(tasks...) } - lTask := tasks[0].convertToRootTask(p.ctx) - rTask := tasks[1].convertToRootTask(p.ctx) + lTask := tasks[0].convertToRootTask(p.SCtx()) + rTask := tasks[1].convertToRootTask(p.SCtx()) p.SetChildren(lTask.plan(), rTask.plan()) task := &rootTask{ p: p, @@ -393,7 +394,7 @@ func negotiateCommonType(lType, rType *types.FieldType) (*types.FieldType, bool, func getProj(ctx sessionctx.Context, p PhysicalPlan) *PhysicalProjection { proj := PhysicalProjection{ Exprs: make([]expression.Expression, 0, len(p.Schema().Columns)), - }.Init(ctx, p.statsInfo(), p.SelectBlockOffset()) + }.Init(ctx, p.StatsInfo(), p.SelectBlockOffset()) for _, col := range p.Schema().Columns { proj.Exprs = append(proj.Exprs, col) } @@ -406,7 +407,7 @@ func appendExpr(p *PhysicalProjection, expr expression.Expression) *expression.C p.Exprs = append(p.Exprs, expr) col := &expression.Column{ - UniqueID: p.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: p.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: expr.GetType(), } col.SetCoercibility(expr.Coercibility()) @@ -451,11 +452,11 @@ func (p *PhysicalHashJoin) convertPartitionKeysIfNeed(lTask, rTask *mppTask) (*m } var lProj, rProj *PhysicalProjection if lChanged { - lProj = getProj(p.ctx, lp) + lProj = getProj(p.SCtx(), lp) lp = lProj } if rChanged { - rProj = getProj(p.ctx, rp) + rProj = getProj(p.SCtx(), rp) rp = rProj } @@ -467,13 +468,13 @@ func (p *PhysicalHashJoin) convertPartitionKeysIfNeed(lTask, rTask *mppTask) (*m if lMask[i] { cType := cTypes[i].Clone() cType.SetFlag(lKey.Col.RetType.GetFlag()) - lCast := expression.BuildCastFunction(p.ctx, lKey.Col, cType) + lCast := expression.BuildCastFunction(p.SCtx(), lKey.Col, cType) lKey = &property.MPPPartitionColumn{Col: appendExpr(lProj, lCast), CollateID: lKey.CollateID} } if rMask[i] { cType := cTypes[i].Clone() cType.SetFlag(rKey.Col.RetType.GetFlag()) - rCast := expression.BuildCastFunction(p.ctx, rKey.Col, cType) + rCast := expression.BuildCastFunction(p.SCtx(), rKey.Col, cType) rKey = &property.MPPPartitionColumn{Col: appendExpr(rProj, rCast), CollateID: rKey.CollateID} } lPartKeys = append(lPartKeys, lKey) @@ -570,8 +571,8 @@ func (p *PhysicalHashJoin) attach2TaskForTiFlash(tasks ...task) task { } func (p *PhysicalMergeJoin) attach2Task(tasks ...task) task { - lTask := tasks[0].convertToRootTask(p.ctx) - rTask := tasks[1].convertToRootTask(p.ctx) + lTask := tasks[0].convertToRootTask(p.SCtx()) + rTask := tasks[1].convertToRootTask(p.SCtx()) p.SetChildren(lTask.plan(), rTask.plan()) t := &rootTask{ p: p, @@ -591,8 +592,7 @@ func buildIndexLookUpTask(ctx sessionctx.Context, t *copTask) *rootTask { }.Init(ctx, t.tablePlan.SelectBlockOffset()) p.PartitionInfo = t.partitionInfo setTableScanToTableRowIDScan(p.tablePlan) - p.stats = t.tablePlan.statsInfo() - + p.SetStats(t.tablePlan.StatsInfo()) // Do not inject the extra Projection even if t.needExtraProj is set, or the schema between the phase-1 agg and // the final agg would be broken. Please reference comments for the similar logic in // (*copTask).convertToRootTaskImpl() for the PhysicalTableReader case. @@ -605,7 +605,7 @@ func buildIndexLookUpTask(ctx sessionctx.Context, t *copTask) *rootTask { if t.needExtraProj && !aggPushedDown { schema := t.originSchema - proj := PhysicalProjection{Exprs: expression.Column2Exprs(schema.Columns)}.Init(ctx, p.stats, t.tablePlan.SelectBlockOffset(), nil) + proj := PhysicalProjection{Exprs: expression.Column2Exprs(schema.Columns)}.Init(ctx, p.StatsInfo(), t.tablePlan.SelectBlockOffset(), nil) proj.SetSchema(schema) proj.SetChildren(p) newTask.p = proj @@ -621,7 +621,7 @@ func extractRows(p PhysicalPlan) float64 { if len(c.Children()) != 0 { f += extractRows(c) } else { - f += c.statsInfo().RowCount + f += c.StatsInfo().RowCount } } return f @@ -699,7 +699,7 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { t.handleRootTaskConds(ctx, newTask) if t.needExtraProj { schema := t.originSchema - proj := PhysicalProjection{Exprs: expression.Column2Exprs(schema.Columns)}.Init(ctx, p.stats, t.idxMergePartPlans[0].SelectBlockOffset(), nil) + proj := PhysicalProjection{Exprs: expression.Column2Exprs(schema.Columns)}.Init(ctx, p.StatsInfo(), t.idxMergePartPlans[0].SelectBlockOffset(), nil) proj.SetSchema(schema) proj.SetChildren(p) newTask.p = proj @@ -711,7 +711,7 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { } else if t.indexPlan != nil { p := PhysicalIndexReader{indexPlan: t.indexPlan}.Init(ctx, t.indexPlan.SelectBlockOffset()) p.PartitionInfo = t.partitionInfo - p.stats = t.indexPlan.statsInfo() + p.SetStats(t.indexPlan.StatsInfo()) newTask.p = p } else { tp := t.tablePlan @@ -730,7 +730,7 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { IsCommonHandle: ts.Table.IsCommonHandle, }.Init(ctx, t.tablePlan.SelectBlockOffset()) p.PartitionInfo = t.partitionInfo - p.stats = t.tablePlan.statsInfo() + p.SetStats(t.tablePlan.StatsInfo()) // If agg was pushed down in attach2Task(), the partial agg was placed on the top of tablePlan, the final agg was // placed above the PhysicalTableReader, and the schema should have been set correctly for them, the schema of @@ -745,7 +745,7 @@ func (t *copTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { } if t.needExtraProj && !aggPushedDown { - proj := PhysicalProjection{Exprs: expression.Column2Exprs(t.originSchema.Columns)}.Init(ts.ctx, ts.stats, ts.SelectBlockOffset(), nil) + proj := PhysicalProjection{Exprs: expression.Column2Exprs(t.originSchema.Columns)}.Init(ts.SCtx(), ts.StatsInfo(), ts.SelectBlockOffset(), nil) proj.SetSchema(t.originSchema) proj.SetChildren(p) newTask.p = proj @@ -765,7 +765,7 @@ func (t *copTask) handleRootTaskConds(ctx sessionctx.Context, newTask *rootTask) logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor } - sel := PhysicalSelection{Conditions: t.rootTaskConds}.Init(ctx, newTask.p.statsInfo().Scale(selectivity), newTask.p.SelectBlockOffset()) + sel := PhysicalSelection{Conditions: t.rootTaskConds}.Init(ctx, newTask.p.StatsInfo().Scale(selectivity), newTask.p.SelectBlockOffset()) sel.fromDataSource = true sel.SetChildren(newTask.p) newTask.p = sel @@ -797,7 +797,7 @@ func (t *rootTask) copy() task { } func (t *rootTask) count() float64 { - return t.p.statsInfo().RowCount + return t.p.StatsInfo().RowCount } func (t *rootTask) plan() PhysicalPlan { @@ -832,16 +832,16 @@ func (p *PhysicalLimit) attach2Task(tasks ...task) task { if (!cop.keepOrder || !cop.indexPlanFinished || cop.indexPlan == nil) && len(cop.rootTaskConds) == 0 { // When limit is pushed down, we should remove its offset. newCount := p.Offset + p.Count - childProfile := cop.plan().statsInfo() + childProfile := cop.plan().StatsInfo() // Strictly speaking, for the row count of stats, we should multiply newCount with "regionNum", // but "regionNum" is unknown since the copTask can be a double read, so we ignore it now. stats := deriveLimitStats(childProfile, float64(newCount)) - pushedDownLimit := PhysicalLimit{PartitionBy: newPartitionBy, Count: newCount}.Init(p.ctx, stats, p.blockOffset) + pushedDownLimit := PhysicalLimit{PartitionBy: newPartitionBy, Count: newCount}.Init(p.SCtx(), stats, p.SelectBlockOffset()) cop = attachPlan2Task(pushedDownLimit, cop).(*copTask) // Don't use clone() so that Limit and its children share the same schema. Otherwise the virtual generated column may not be resolved right. pushedDownLimit.SetSchema(pushedDownLimit.children[0].Schema()) } - t = cop.convertToRootTask(p.ctx) + t = cop.convertToRootTask(p.SCtx()) sunk = p.sinkIntoIndexLookUp(t) } else if !cop.idxMergeIsIntersection { // We only support push part of the order prop down to index merge case. @@ -849,28 +849,28 @@ func (p *PhysicalLimit) attach2Task(tasks ...task) task { newCount := p.Offset + p.Count limitChildren := make([]PhysicalPlan, 0, len(cop.idxMergePartPlans)) for _, partialScan := range cop.idxMergePartPlans { - childProfile := partialScan.statsInfo() + childProfile := partialScan.StatsInfo() stats := deriveLimitStats(childProfile, float64(newCount)) - pushedDownLimit := PhysicalLimit{PartitionBy: newPartitionBy, Count: newCount}.Init(p.ctx, stats, p.blockOffset) + pushedDownLimit := PhysicalLimit{PartitionBy: newPartitionBy, Count: newCount}.Init(p.SCtx(), stats, p.SelectBlockOffset()) pushedDownLimit.SetChildren(partialScan) pushedDownLimit.SetSchema(pushedDownLimit.children[0].Schema()) limitChildren = append(limitChildren, pushedDownLimit) } cop.idxMergePartPlans = limitChildren } - t = cop.convertToRootTask(p.ctx) + t = cop.convertToRootTask(p.SCtx()) } else { // Whatever the remained case is, we directly convert to it to root task. - t = cop.convertToRootTask(p.ctx) + t = cop.convertToRootTask(p.SCtx()) } } else if mpp, ok := t.(*mppTask); ok { newCount := p.Offset + p.Count - childProfile := mpp.plan().statsInfo() + childProfile := mpp.plan().StatsInfo() stats := deriveLimitStats(childProfile, float64(newCount)) - pushedDownLimit := PhysicalLimit{Count: newCount, PartitionBy: newPartitionBy}.Init(p.ctx, stats, p.blockOffset) + pushedDownLimit := PhysicalLimit{Count: newCount, PartitionBy: newPartitionBy}.Init(p.SCtx(), stats, p.SelectBlockOffset()) mpp = attachPlan2Task(pushedDownLimit, mpp).(*mppTask) pushedDownLimit.SetSchema(pushedDownLimit.children[0].Schema()) - t = mpp.convertToRootTask(p.ctx) + t = mpp.convertToRootTask(p.SCtx()) } if sunk { return t @@ -911,7 +911,7 @@ func (p *PhysicalLimit) sinkIntoIndexLookUp(t task) bool { if p.Schema().Len() != reader.Schema().Len() { extraProj := PhysicalProjection{ Exprs: expression.Column2Exprs(p.schema.Columns), - }.Init(p.SCtx(), p.statsInfo(), p.blockOffset, nil) + }.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), nil) extraProj.SetSchema(p.schema) // If the root.p is already a Projection. We left the optimization for the later Projection Elimination. extraProj.SetChildren(root.p) @@ -922,15 +922,15 @@ func (p *PhysicalLimit) sinkIntoIndexLookUp(t task) bool { Offset: p.Offset, Count: p.Count, } - originStats := ts.stats - ts.stats = p.stats + originStats := ts.StatsInfo() + ts.SetStats(p.StatsInfo()) if originStats != nil { // keep the original stats version - ts.stats.StatsVersion = originStats.StatsVersion + ts.StatsInfo().StatsVersion = originStats.StatsVersion } - reader.stats = p.stats + reader.SetStats(p.StatsInfo()) if isProj { - proj.stats = p.stats + proj.SetStats(p.StatsInfo()) } return true } @@ -960,7 +960,7 @@ func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { newPartitionBy = append(newPartitionBy, expr.Clone()) } newCount := p.Offset + p.Count - childProfile := childPlan.statsInfo() + childProfile := childPlan.StatsInfo() // Strictly speaking, for the row count of pushed down TopN, we should multiply newCount with "regionNum", // but "regionNum" is unknown since the copTask can be a double read, so we ignore it now. stats := deriveLimitStats(childProfile, float64(newCount)) @@ -968,7 +968,7 @@ func (p *PhysicalTopN) getPushedDownTopN(childPlan PhysicalPlan) *PhysicalTopN { ByItems: newByItems, PartitionBy: newPartitionBy, Count: newCount, - }.Init(p.ctx, stats, p.blockOffset, p.GetChildReqProps(0)) + }.Init(p.SCtx(), stats, p.SelectBlockOffset(), p.GetChildReqProps(0)) topN.SetChildren(childPlan) return topN } @@ -1003,7 +1003,7 @@ func (p *PhysicalTopN) canExpressionConvertedToPB(storeTp kv.StoreType) bool { for _, item := range p.ByItems { exprs = append(exprs, item.Expr) } - return expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, exprs, p.ctx.GetClient(), storeTp) + return expression.CanExprsPushDown(p.SCtx().GetSessionVars().StmtCtx, exprs, p.SCtx().GetClient(), storeTp) } // containVirtualColumn checks whether TopN.ByItems contains virtual generated columns. @@ -1081,7 +1081,7 @@ func (p *PhysicalTopN) attach2Task(tasks ...task) task { pushedDownTopN := p.getPushedDownTopN(mppTask.p) mppTask.p = pushedDownTopN } - rootTask := t.convertToRootTask(p.ctx) + rootTask := t.convertToRootTask(p.SCtx()) // Skip TopN with partition on the root. This is a derived topN and window function // will take care of the filter. if len(p.GetPartitionBy()) > 0 { @@ -1140,19 +1140,19 @@ func (p *PhysicalTopN) pushPartialTopNDownToCop(copTsk *copTask) (task, bool) { partialSel, _ = finalScan.(*PhysicalSelection) finalScan = finalScan.Children()[0] } - if partialSel != nil && finalScan.statsInfo().RowCount > 0 { - selSelectivityOnPartialScan[i] = partialSel.statsInfo().RowCount / finalScan.statsInfo().RowCount + if partialSel != nil && finalScan.StatsInfo().RowCount > 0 { + selSelectivityOnPartialScan[i] = partialSel.StatsInfo().RowCount / finalScan.StatsInfo().RowCount } if plan, ok := finalScan.(*PhysicalTableScan); ok { plan.ByItems = p.ByItems - if plan.Table.GetPartitionInfo() != nil && p.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { - plan.Columns, plan.schema, _ = AddExtraPhysTblIDColumn(plan.ctx, plan.Columns, plan.Schema()) + if plan.Table.GetPartitionInfo() != nil && p.SCtx().GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { + plan.Columns, plan.schema, _ = AddExtraPhysTblIDColumn(plan.SCtx(), plan.Columns, plan.Schema()) } } if plan, ok := finalScan.(*PhysicalIndexScan); ok { plan.ByItems = p.ByItems - if plan.Table.GetPartitionInfo() != nil && p.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() && !plan.Index.Global { - plan.Columns, plan.schema, _ = AddExtraPhysTblIDColumn(plan.ctx, plan.Columns, plan.Schema()) + if plan.Table.GetPartitionInfo() != nil && p.SCtx().GetSessionVars().StmtCtx.UseDynamicPartitionPrune() && !plan.Index.Global { + plan.Columns, plan.schema, _ = AddExtraPhysTblIDColumn(plan.SCtx(), plan.Columns, plan.Schema()) } } partialScans = append(partialScans, finalScan) @@ -1177,7 +1177,7 @@ func (p *PhysicalTopN) pushPartialTopNDownToCop(copTsk *copTask) (task, bool) { return nil, false } clonedTblScan := cloned.(*PhysicalTableScan) - clonedTblScan.statsInfo().ScaleByExpectCnt(float64(p.Count+p.Offset) * float64(len(copTsk.idxMergePartPlans))) + clonedTblScan.StatsInfo().ScaleByExpectCnt(float64(p.Count+p.Offset) * float64(len(copTsk.idxMergePartPlans))) tblInfo := clonedTblScan.Table if tblInfo.PKIsHandle { pk := tblInfo.GetPkColInfo() @@ -1205,9 +1205,9 @@ func (p *PhysicalTopN) pushPartialTopNDownToCop(copTsk *copTask) (task, bool) { } } // global index for tableScan with keepOrder also need PhysicalTblID - if clonedTblScan.Table.GetPartitionInfo() != nil && p.ctx.GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { + if clonedTblScan.Table.GetPartitionInfo() != nil && p.SCtx().GetSessionVars().StmtCtx.UseDynamicPartitionPrune() { var succ bool - clonedTblScan.Columns, clonedTblScan.schema, succ = AddExtraPhysTblIDColumn(clonedTblScan.ctx, clonedTblScan.Columns, clonedTblScan.Schema()) + clonedTblScan.Columns, clonedTblScan.schema, succ = AddExtraPhysTblIDColumn(clonedTblScan.SCtx(), clonedTblScan.Columns, clonedTblScan.Schema()) copTsk.needExtraProj = copTsk.needExtraProj || succ } clonedTblScan.HandleCols, err = clonedTblScan.HandleCols.ResolveIndices(clonedTblScan.Schema()) @@ -1219,7 +1219,7 @@ func (p *PhysicalTopN) pushPartialTopNDownToCop(copTsk *copTask) (task, bool) { } copTsk.tablePlan = clonedTblScan copTsk.indexPlanFinished = true - rootTask := copTsk.convertToRootTask(p.ctx) + rootTask := copTsk.convertToRootTask(p.SCtx()) indexMerge, ok := rootTask.p.(*PhysicalIndexMergeReader) if !ok { // needExtraProj == true @@ -1237,7 +1237,7 @@ func (p *PhysicalTopN) pushPartialTopNDownToCop(copTsk *copTask) (task, bool) { return rootTask, true } - rootTask := copTsk.convertToRootTask(p.ctx) + rootTask := copTsk.convertToRootTask(p.SCtx()) return attachPlan2Task(p, rootTask), true } @@ -1309,7 +1309,7 @@ func (p *PhysicalTopN) checkSubScans(colsProp *property.PhysicalProperty, isDesc func (p *PhysicalTopN) addPartialLimitForSubScans(copSubPlans []PhysicalPlan, finalPartialScans []PhysicalPlan, selSelectivities []float64) []PhysicalPlan { limitAddedPlan := make([]PhysicalPlan, 0, len(copSubPlans)) for i, copSubPlan := range copSubPlans { - childProfile := copSubPlan.statsInfo() + childProfile := copSubPlan.StatsInfo() newCount := p.Offset + p.Count stats := deriveLimitStats(childProfile, float64(newCount)) pushedLimit := PhysicalLimit{ @@ -1320,12 +1320,12 @@ func (p *PhysicalTopN) addPartialLimitForSubScans(copSubPlans []PhysicalPlan, fi // A similar but simplified logic compared the ExpectedCnt handling logic in getOriginalPhysicalIndexScan. child := pushedLimit.Children()[0] // The row count of the direct child of Limit should be adjusted to be no larger than the Limit.Count. - child.SetStats(child.statsInfo().ScaleByExpectCnt(float64(newCount))) + child.SetStats(child.StatsInfo().ScaleByExpectCnt(float64(newCount))) // The Limit->Selection->IndexScan case: // adjust the row count of IndexScan according to the selectivity of the Selection. if selSelectivities[i] > 0 && selSelectivities[i] < 1 { - scaledRowCount := child.Stats().RowCount / selSelectivities[i] - finalPartialScans[i].SetStats(finalPartialScans[i].Stats().ScaleByExpectCnt(scaledRowCount)) + scaledRowCount := child.StatsInfo().RowCount / selSelectivities[i] + finalPartialScans[i].SetStats(finalPartialScans[i].StatsInfo().ScaleByExpectCnt(scaledRowCount)) } limitAddedPlan = append(limitAddedPlan, pushedLimit) } @@ -1346,18 +1346,18 @@ func (p *PhysicalExpand) attach2Task(tasks ...task) task { func (p *PhysicalProjection) attach2Task(tasks ...task) task { t := tasks[0].copy() if cop, ok := t.(*copTask); ok { - if (len(cop.rootTaskConds) == 0 && len(cop.idxMergePartPlans) == 0) && expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, p.Exprs, p.ctx.GetClient(), cop.getStoreType()) { + if (len(cop.rootTaskConds) == 0 && len(cop.idxMergePartPlans) == 0) && expression.CanExprsPushDown(p.SCtx().GetSessionVars().StmtCtx, p.Exprs, p.SCtx().GetClient(), cop.getStoreType()) { copTask := attachPlan2Task(p, cop) return copTask } } else if mpp, ok := t.(*mppTask); ok { - if expression.CanExprsPushDown(p.ctx.GetSessionVars().StmtCtx, p.Exprs, p.ctx.GetClient(), kv.TiFlash) { + if expression.CanExprsPushDown(p.SCtx().GetSessionVars().StmtCtx, p.Exprs, p.SCtx().GetClient(), kv.TiFlash) { p.SetChildren(mpp.p) mpp.p = p return mpp } } - t = t.convertToRootTask(p.ctx) + t = t.convertToRootTask(p.SCtx()) t = attachPlan2Task(p, t) if root, ok := tasks[0].(*rootTask); ok && root.isEmpty { t.(*rootTask).isEmpty = true @@ -1399,7 +1399,7 @@ func (p *PhysicalUnionAll) attach2Task(tasks ...task) task { t := &rootTask{p: p} childPlans := make([]PhysicalPlan, 0, len(tasks)) for _, task := range tasks { - task = task.convertToRootTask(p.ctx) + task = task.convertToRootTask(p.SCtx()) childPlans = append(childPlans, task.plan()) } p.SetChildren(childPlans...) @@ -1408,12 +1408,12 @@ func (p *PhysicalUnionAll) attach2Task(tasks ...task) task { func (sel *PhysicalSelection) attach2Task(tasks ...task) task { if mppTask, _ := tasks[0].(*mppTask); mppTask != nil { // always push to mpp task. - sc := sel.ctx.GetSessionVars().StmtCtx - if expression.CanExprsPushDown(sc, sel.Conditions, sel.ctx.GetClient(), kv.TiFlash) { + sc := sel.SCtx().GetSessionVars().StmtCtx + if expression.CanExprsPushDown(sc, sel.Conditions, sel.SCtx().GetClient(), kv.TiFlash) { return attachPlan2Task(sel, mppTask.copy()) } } - t := tasks[0].convertToRootTask(sel.ctx) + t := tasks[0].convertToRootTask(sel.SCtx()) return attachPlan2Task(sel, t) } @@ -1766,7 +1766,7 @@ func (p *basePhysicalAgg) convertAvgForMPP() *PhysicalProjection { // inset a count(column) avgCount := aggFunc.Clone() avgCount.Name = ast.AggFuncCount - err := avgCount.TypeInfer(p.ctx) + err := avgCount.TypeInfer(p.SCtx()) if err != nil { // must not happen return nil } @@ -1787,9 +1787,9 @@ func (p *basePhysicalAgg) convertAvgForMPP() *PhysicalProjection { } newSchema.Append(avgSumCol) // avgSumCol/(case when avgCountCol=0 then 1 else avgCountCol end) - eq := expression.NewFunctionInternal(p.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), avgCountCol, expression.NewZero()) - caseWhen := expression.NewFunctionInternal(p.ctx, ast.Case, avgCountCol.RetType, eq, expression.NewOne(), avgCountCol) - divide := expression.NewFunctionInternal(p.ctx, ast.Div, avgSumCol.RetType, avgSumCol, caseWhen) + eq := expression.NewFunctionInternal(p.SCtx(), ast.EQ, types.NewFieldType(mysql.TypeTiny), avgCountCol, expression.NewZero()) + caseWhen := expression.NewFunctionInternal(p.SCtx(), ast.Case, avgCountCol.RetType, eq, expression.NewOne(), avgCountCol) + divide := expression.NewFunctionInternal(p.SCtx(), ast.Div, avgSumCol.RetType, avgSumCol, caseWhen) divide.(*expression.ScalarFunction).RetType = p.schema.Columns[i].RetType exprs = append(exprs, divide) } else { @@ -1812,7 +1812,7 @@ func (p *basePhysicalAgg) convertAvgForMPP() *PhysicalProjection { Exprs: exprs, CalculateNoDelay: false, AvoidColumnEvaluator: false, - }.Init(p.SCtx(), p.stats, p.SelectBlockOffset(), p.GetChildReqProps(0).CloneEssentialFields()) + }.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), p.GetChildReqProps(0).CloneEssentialFields()) proj.SetSchema(p.schema) p.AggFuncs = newAggFuncs @@ -1823,10 +1823,10 @@ func (p *basePhysicalAgg) convertAvgForMPP() *PhysicalProjection { func (p *basePhysicalAgg) newPartialAggregate(copTaskType kv.StoreType, isMPPTask bool) (partial, final PhysicalPlan) { // Check if this aggregation can push down. - if !CheckAggCanPushCop(p.ctx, p.AggFuncs, p.GroupByItems, copTaskType) { + if !CheckAggCanPushCop(p.SCtx(), p.AggFuncs, p.GroupByItems, copTaskType) { return nil, p.self } - partialPref, finalPref, firstRowFuncMap := BuildFinalModeAggregation(p.ctx, &AggInfo{ + partialPref, finalPref, firstRowFuncMap := BuildFinalModeAggregation(p.SCtx(), &AggInfo{ AggFuncs: p.AggFuncs, GroupByItems: p.GroupByItems, Schema: p.Schema().Clone(), @@ -1834,17 +1834,17 @@ func (p *basePhysicalAgg) newPartialAggregate(copTaskType kv.StoreType, isMPPTas if partialPref == nil { return nil, p.self } - if p.tp == plancodec.TypeStreamAgg && len(partialPref.GroupByItems) != len(finalPref.GroupByItems) { + if p.TP() == plancodec.TypeStreamAgg && len(partialPref.GroupByItems) != len(finalPref.GroupByItems) { return nil, p.self } // Remove unnecessary FirstRow. - partialPref.AggFuncs = RemoveUnnecessaryFirstRow(p.ctx, + partialPref.AggFuncs = RemoveUnnecessaryFirstRow(p.SCtx(), finalPref.GroupByItems, partialPref.AggFuncs, partialPref.GroupByItems, partialPref.Schema, firstRowFuncMap) if copTaskType == kv.TiDB { // For partial agg of TiDB cop task, since TiDB coprocessor reuse the TiDB executor, // and TiDB aggregation executor won't output the group by value, // so we need add `firstrow` aggregation function to output the group by value. - aggFuncs, err := genFirstRowAggForGroupBy(p.ctx, partialPref.GroupByItems) + aggFuncs, err := genFirstRowAggForGroupBy(p.SCtx(), partialPref.GroupByItems) if err != nil { return nil, p.self } @@ -1856,12 +1856,12 @@ func (p *basePhysicalAgg) newPartialAggregate(copTaskType kv.StoreType, isMPPTas partialAgg := p.self // Create physical "final" aggregation. prop := &property.PhysicalProperty{ExpectedCnt: math.MaxFloat64} - if p.tp == plancodec.TypeStreamAgg { + if p.TP() == plancodec.TypeStreamAgg { finalAgg := basePhysicalAgg{ AggFuncs: finalPref.AggFuncs, GroupByItems: finalPref.GroupByItems, MppRunMode: p.MppRunMode, - }.initForStream(p.ctx, p.stats, p.blockOffset, prop) + }.initForStream(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), prop) finalAgg.schema = finalPref.Schema return partialAgg, finalAgg } @@ -1870,7 +1870,7 @@ func (p *basePhysicalAgg) newPartialAggregate(copTaskType kv.StoreType, isMPPTas AggFuncs: finalPref.AggFuncs, GroupByItems: finalPref.GroupByItems, MppRunMode: p.MppRunMode, - }.initForHash(p.ctx, p.stats, p.blockOffset, prop) + }.initForHash(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset(), prop) finalAgg.schema = finalPref.Schema // partialAgg and finalAgg use the same ref of stats return partialAgg, finalAgg @@ -1885,7 +1885,7 @@ func (p *basePhysicalAgg) scale3StageForDistinctAgg() (bool, expression.Grouping // canUse3Stage4MultiDistinctAgg returns true if this agg can use 3 stage for multi distinct aggregation func (p *basePhysicalAgg) canUse3Stage4MultiDistinctAgg() (can bool, gss expression.GroupingSets) { - if !p.ctx.GetSessionVars().Enable3StageDistinctAgg || !p.ctx.GetSessionVars().Enable3StageMultiDistinctAgg || len(p.GroupByItems) > 0 { + if !p.SCtx().GetSessionVars().Enable3StageDistinctAgg || !p.SCtx().GetSessionVars().Enable3StageMultiDistinctAgg || len(p.GroupByItems) > 0 { return false, nil } defer func() { @@ -1956,7 +1956,7 @@ func (p *basePhysicalAgg) canUse3Stage4MultiDistinctAgg() (can bool, gss express // canUse3Stage4SingleDistinctAgg returns true if this agg can use 3 stage for distinct aggregation func (p *basePhysicalAgg) canUse3Stage4SingleDistinctAgg() bool { num := 0 - if !p.ctx.GetSessionVars().Enable3StageDistinctAgg || len(p.GroupByItems) > 0 { + if !p.SCtx().GetSessionVars().Enable3StageDistinctAgg || len(p.GroupByItems) > 0 { return false } for _, fun := range p.AggFuncs { @@ -2065,7 +2065,7 @@ func (p *PhysicalStreamAgg) attach2Task(tasks ...task) task { // 2. the case that there's filters should be calculated on TiDB side. // 3. the case of index merge if (cop.indexPlan != nil && cop.tablePlan != nil && cop.keepOrder) || len(cop.rootTaskConds) > 0 || len(cop.idxMergePartPlans) > 0 { - t = cop.convertToRootTask(p.ctx) + t = cop.convertToRootTask(p.SCtx()) attachPlan2Task(p, t) } else { storeType := cop.getStoreType() @@ -2091,11 +2091,11 @@ func (p *PhysicalStreamAgg) attach2Task(tasks ...task) task { cop.indexPlan = partialAgg } } - t = cop.convertToRootTask(p.ctx) + t = cop.convertToRootTask(p.SCtx()) attachPlan2Task(finalAgg, t) } } else if mpp, ok := t.(*mppTask); ok { - t = mpp.convertToRootTask(p.ctx) + t = mpp.convertToRootTask(p.SCtx()) attachPlan2Task(p, t) } else { attachPlan2Task(p, t) @@ -2109,7 +2109,7 @@ func (p *PhysicalHashAgg) cpuCostDivisor(hasDistinct bool) (divisor, con float64 if hasDistinct { return 0, 0 } - sessionVars := p.ctx.GetSessionVars() + sessionVars := p.SCtx().GetSessionVars() finalCon, partialCon := sessionVars.HashAggFinalConcurrency(), sessionVars.HashAggPartialConcurrency() // According to `ValidateSetSystemVar`, `finalCon` and `partialCon` cannot be less than or equal to 0. if finalCon == 1 && partialCon == 1 { @@ -2187,9 +2187,9 @@ func (p *PhysicalHashAgg) scaleStats4GroupingSets(groupingSets expression.Groupi } // After group operator, all same rows are grouped into one row, that means all // change the sub-agg's stats - if p.stats != nil { + if p.StatsInfo() != nil { // equivalence to a new cloned one. (cause finalAgg and partialAgg may share a same copy of stats) - cpStats := p.stats.Scale(1) + cpStats := p.StatsInfo().Scale(1) cpStats.RowCount = sumNDV // We cannot estimate the ColNDVs for every output, so we use a conservative strategy. for k := range cpStats.ColNDVs { @@ -2219,7 +2219,7 @@ func (p *PhysicalHashAgg) scaleStats4GroupingSets(groupingSets expression.Groupi } oneGNDV.NDV = newGNDV } - p.stats = cpStats + p.SetStats(cpStats) } } @@ -2267,7 +2267,7 @@ func (p *PhysicalHashAgg) adjust3StagePhaseAgg(partialAgg, finalAgg PhysicalPlan schemaMap := make(map[int64]*expression.Column, len(middleHashAgg.AggFuncs)) for i, fun := range middleHashAgg.AggFuncs { col := &expression.Column{ - UniqueID: p.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: p.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: fun.RetTp, } if fun.HasDistinct { @@ -2316,16 +2316,16 @@ func (p *PhysicalHashAgg) adjust3StagePhaseAgg(partialAgg, finalAgg PhysicalPlan // enforce Expand operator above the children. // physical plan is enumerated without children from itself, use mpp subtree instead p.children. // scale(len(groupingSets)) will change the NDV, while Expand doesn't change the NDV and groupNDV. - stats := mpp.p.statsInfo().Scale(float64(1)) + stats := mpp.p.StatsInfo().Scale(float64(1)) stats.RowCount = stats.RowCount * float64(len(groupingSets)) physicalExpand := PhysicalExpand{ GroupingSets: groupingSets, - }.Init(p.ctx, stats, mpp.p.SelectBlockOffset()) + }.Init(p.SCtx(), stats, mpp.p.SelectBlockOffset()) // generate a new column as groupingID to identify which this row is targeting for. tp := types.NewFieldType(mysql.TypeLonglong) tp.SetFlag(mysql.UnsignedFlag | mysql.NotNullFlag) groupingIDCol = &expression.Column{ - UniqueID: p.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: p.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: tp, } // append the physical expand op with groupingID column. @@ -2342,15 +2342,15 @@ func (p *PhysicalHashAgg) adjust3StagePhaseAgg(partialAgg, finalAgg PhysicalPlan } cloneHashAgg := clonedAgg.(*PhysicalHashAgg) // Clone(), it will share same base-plan elements from the finalAgg, including id,tp,stats. Make a new one here. - cloneHashAgg.basePlan = newBasePlan(cloneHashAgg.ctx, cloneHashAgg.tp, cloneHashAgg.blockOffset) - cloneHashAgg.stats = finalAgg.Stats() // reuse the final agg stats here. + cloneHashAgg.Plan = base.NewBasePlan(cloneHashAgg.SCtx(), cloneHashAgg.TP(), cloneHashAgg.SelectBlockOffset()) + cloneHashAgg.SetStats(finalAgg.StatsInfo()) // reuse the final agg stats here. // step1: adjust partial agg, for normal agg here, adjust it to target for specified group data. // Since we may substitute the first arg of normal agg with case-when expression here, append a // customized proj here rather than depending on postOptimize to insert a blunt one for us. // // proj4Partial output all the base col from lower op + caseWhen proj cols. - proj4Partial := new(PhysicalProjection).Init(p.ctx, mpp.p.statsInfo(), mpp.p.SelectBlockOffset()) + proj4Partial := new(PhysicalProjection).Init(p.SCtx(), mpp.p.StatsInfo(), mpp.p.SelectBlockOffset()) for _, col := range mpp.p.Schema().Columns { proj4Partial.Exprs = append(proj4Partial.Exprs, col) } @@ -2360,15 +2360,15 @@ func (p *PhysicalHashAgg) adjust3StagePhaseAgg(partialAgg, finalAgg PhysicalPlan partialHashAgg.GroupByItems = append(partialHashAgg.GroupByItems, groupingIDCol) partialHashAgg.schema.Append(groupingIDCol.(*expression.Column)) // it will create a new stats for partial agg. - partialHashAgg.scaleStats4GroupingSets(groupingSets, groupingIDCol.(*expression.Column), proj4Partial.Schema(), proj4Partial.statsInfo()) + partialHashAgg.scaleStats4GroupingSets(groupingSets, groupingIDCol.(*expression.Column), proj4Partial.Schema(), proj4Partial.StatsInfo()) for _, fun := range partialHashAgg.AggFuncs { if !fun.HasDistinct { // for normal agg phase1, we should also modify them to target for specified group data. // Expr = (case when groupingID = targeted_groupingID then arg else null end) - eqExpr := expression.NewFunctionInternal(p.ctx, ast.EQ, types.NewFieldType(mysql.TypeTiny), groupingIDCol, expression.NewUInt64Const(fun.GroupingID)) - caseWhen := expression.NewFunctionInternal(p.ctx, ast.Case, fun.Args[0].GetType(), eqExpr, fun.Args[0], expression.NewNull()) + eqExpr := expression.NewFunctionInternal(p.SCtx(), ast.EQ, types.NewFieldType(mysql.TypeTiny), groupingIDCol, expression.NewUInt64Const(fun.GroupingID)) + caseWhen := expression.NewFunctionInternal(p.SCtx(), ast.Case, fun.Args[0].GetType(), eqExpr, fun.Args[0], expression.NewNull()) caseWhenProjCol := &expression.Column{ - UniqueID: p.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: p.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: fun.Args[0].GetType(), } proj4Partial.Exprs = append(proj4Partial.Exprs, caseWhen) @@ -2384,7 +2384,7 @@ func (p *PhysicalHashAgg) adjust3StagePhaseAgg(partialAgg, finalAgg PhysicalPlan schemaMap := make(map[int64]*expression.Column, len(middleHashAgg.AggFuncs)) for _, fun := range middleHashAgg.AggFuncs { col := &expression.Column{ - UniqueID: p.ctx.GetSessionVars().AllocPlanColumnID(), + UniqueID: p.SCtx().GetSessionVars().AllocPlanColumnID(), RetType: fun.RetTp, } if fun.HasDistinct { @@ -2486,7 +2486,7 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { if partialAgg != nil { attachPlan2Task(partialAgg, mpp) } - t = mpp.convertToRootTask(p.ctx) + t = mpp.convertToRootTask(p.SCtx()) attachPlan2Task(finalAgg, t) return t case MppScalar: @@ -2545,7 +2545,7 @@ func (p *PhysicalHashAgg) attach2TaskForMpp(tasks ...task) task { if proj == nil { proj = PhysicalProjection{ Exprs: make([]expression.Expression, 0, len(p.Schema().Columns)), - }.Init(p.ctx, p.statsInfo(), p.SelectBlockOffset()) + }.Init(p.SCtx(), p.StatsInfo(), p.SelectBlockOffset()) for _, col := range p.Schema().Columns { proj.Exprs = append(proj.Exprs, col) } @@ -2591,10 +2591,10 @@ func (p *PhysicalHashAgg) attach2Task(tasks ...task) task { // column may be independent of the column used for region distribution, so a closer // estimation of network cost for hash aggregation may multiply the number of // regions involved in the `partialAgg`, which is unknown however. - t = cop.convertToRootTask(p.ctx) + t = cop.convertToRootTask(p.SCtx()) attachPlan2Task(finalAgg, t) } else { - t = cop.convertToRootTask(p.ctx) + t = cop.convertToRootTask(p.SCtx()) attachPlan2Task(p, t) } } else if _, ok := t.(*mppTask); ok { @@ -2625,7 +2625,7 @@ func (p *PhysicalWindow) attach2Task(tasks ...task) task { if mpp, ok := tasks[0].copy().(*mppTask); ok && p.storeTp == kv.TiFlash { return p.attach2TaskForMPP(mpp) } - t := tasks[0].convertToRootTask(p.ctx) + t := tasks[0].convertToRootTask(p.SCtx()) return attachPlan2Task(p.self, t) } @@ -2640,7 +2640,7 @@ func (p *PhysicalCTEStorage) attach2Task(tasks ...task) task { tblColHists: mpp.tblColHists, } } - t.convertToRootTask(p.ctx) + t.convertToRootTask(p.SCtx()) p.SetChildren(t.plan()) return &rootTask{ p: p, @@ -2699,7 +2699,7 @@ type mppTask struct { } func (t *mppTask) count() float64 { - return t.p.statsInfo().RowCount + return t.p.StatsInfo().RowCount } func (t *mppTask) copy() task { @@ -2744,8 +2744,8 @@ func collectPartitionInfosFromMPPPlan(p *PhysicalTableReader, mppPlan PhysicalPl } func collectRowSizeFromMPPPlan(mppPlan PhysicalPlan) (rowSize float64) { - if mppPlan != nil && mppPlan.Stats() != nil && mppPlan.Stats().HistColl != nil { - return mppPlan.Stats().HistColl.GetAvgRowSize(mppPlan.SCtx(), mppPlan.Schema().Columns, false, false) + if mppPlan != nil && mppPlan.StatsInfo() != nil && mppPlan.StatsInfo().HistColl != nil { + return mppPlan.StatsInfo().HistColl.GetAvgRowSize(mppPlan.SCtx(), mppPlan.Schema().Columns, false, false) } return 1 // use 1 as lower-bound for safety } @@ -2775,14 +2775,14 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { tryExpandVirtualColumn(t.p) sender := PhysicalExchangeSender{ ExchangeType: tipb.ExchangeType_PassThrough, - }.Init(ctx, t.p.statsInfo()) + }.Init(ctx, t.p.StatsInfo()) sender.SetChildren(t.p) p := PhysicalTableReader{ tablePlan: sender, StoreType: kv.TiFlash, }.Init(ctx, t.p.SelectBlockOffset()) - p.stats = t.p.statsInfo() + p.SetStats(t.p.StatsInfo()) collectPartitionInfosFromMPPPlan(p, t.p) rt := &rootTask{ p: p, @@ -2808,7 +2808,7 @@ func (t *mppTask) convertToRootTaskImpl(ctx sessionctx.Context) *rootTask { logutil.BgLogger().Debug("calculate selectivity failed, use selection factor", zap.Error(err)) selectivity = SelectionFactor } - sel := PhysicalSelection{Conditions: t.rootTaskConds}.Init(ctx, rt.p.statsInfo().Scale(selectivity), rt.p.SelectBlockOffset()) + sel := PhysicalSelection{Conditions: t.rootTaskConds}.Init(ctx, rt.p.StatsInfo().Scale(selectivity), rt.p.SelectBlockOffset()) sel.fromDataSource = true sel.SetChildren(rt.p) rt.p = sel @@ -2863,14 +2863,14 @@ func (t *mppTask) enforceExchangerImpl(prop *property.PhysicalProperty) *mppTask sender := PhysicalExchangeSender{ ExchangeType: prop.MPPPartitionTp.ToExchangeType(), HashCols: prop.MPPPartitionCols, - }.Init(ctx, t.p.statsInfo()) + }.Init(ctx, t.p.StatsInfo()) if ctx.GetSessionVars().ChooseMppVersion() >= kv.MppVersionV1 { sender.CompressionMode = ctx.GetSessionVars().ChooseMppExchangeCompressionMode() } sender.SetChildren(t.p) - receiver := PhysicalExchangeReceiver{}.Init(ctx, t.p.statsInfo()) + receiver := PhysicalExchangeReceiver{}.Init(ctx, t.p.StatsInfo()) receiver.SetChildren(sender) return &mppTask{ p: receiver, diff --git a/planner/core/tiflash_selection_late_materialization.go b/planner/core/tiflash_selection_late_materialization.go index 4c4f3d46339ea..738e2ff486594 100644 --- a/planner/core/tiflash_selection_late_materialization.go +++ b/planner/core/tiflash_selection_late_materialization.go @@ -217,7 +217,7 @@ func predicatePushDownToTableScanImpl(sctx sessionctx.Context, physicalSelection selectedColumnCount := 0 selectedSelectivity := 1.0 totalColumnCount := len(physicalTableScan.Columns) - tableRowCount := physicalTableScan.stats.RowCount + tableRowCount := physicalTableScan.StatsInfo().RowCount for _, exprGroup := range sortedConds { mergedConds := append(selectedConds, exprGroup.exprs...) @@ -251,5 +251,5 @@ func predicatePushDownToTableScanImpl(sctx sessionctx.Context, physicalSelection // add the pushed down conditions to table scan physicalTableScan.lateMaterializationFilterCondition = selectedConds // Update the row count of table scan after pushing down the conditions. - physicalTableScan.stats.RowCount *= selectedSelectivity + physicalTableScan.StatsInfo().RowCount *= selectedSelectivity } diff --git a/planner/core/util.go b/planner/core/util.go index 3aff180018815..ad93a0fda6fa5 100644 --- a/planner/core/util.go +++ b/planner/core/util.go @@ -24,6 +24,7 @@ import ( "github.com/pingcap/tidb/parser/ast" "github.com/pingcap/tidb/parser/model" "github.com/pingcap/tidb/parser/mysql" + "github.com/pingcap/tidb/planner/core/internal/base" "github.com/pingcap/tidb/sessionctx" "github.com/pingcap/tidb/table/tables" "github.com/pingcap/tidb/types" @@ -197,7 +198,7 @@ func (s *physicalSchemaProducer) MemoryUsage() (sum int64) { type baseSchemaProducer struct { schema *expression.Schema names types.NameSlice - basePlan + base.Plan } // OutputNames returns the outputting names of each column. @@ -233,7 +234,7 @@ func (s *baseSchemaProducer) MemoryUsage() (sum int64) { return } - sum = size.SizeOfPointer + size.SizeOfSlice + int64(cap(s.names))*size.SizeOfPointer + s.basePlan.MemoryUsage() + sum = size.SizeOfPointer + size.SizeOfSlice + int64(cap(s.names))*size.SizeOfPointer + s.Plan.MemoryUsage() if s.schema != nil { sum += s.schema.MemoryUsage() } @@ -295,12 +296,12 @@ func GetStatsInfoFromFlatPlan(flat *FlatPhysicalPlan) map[string]uint64 { for _, op := range flat.Main { switch p := op.Origin.(type) { case *PhysicalIndexScan: - if _, ok := res[p.Table.Name.O]; p.stats != nil && !ok { - res[p.Table.Name.O] = p.stats.StatsVersion + if _, ok := res[p.Table.Name.O]; p.StatsInfo() != nil && !ok { + res[p.Table.Name.O] = p.StatsInfo().StatsVersion } case *PhysicalTableScan: - if _, ok := res[p.Table.Name.O]; p.stats != nil && !ok { - res[p.Table.Name.O] = p.stats.StatsVersion + if _, ok := res[p.Table.Name.O]; p.StatsInfo() != nil && !ok { + res[p.Table.Name.O] = p.StatsInfo().StatsVersion } } } diff --git a/planner/implementation/simple_plans.go b/planner/implementation/simple_plans.go index be5ff50f25d25..12b0e3b14ffe0 100644 --- a/planner/implementation/simple_plans.go +++ b/planner/implementation/simple_plans.go @@ -32,7 +32,7 @@ func NewProjectionImpl(proj *plannercore.PhysicalProjection) *ProjectionImpl { // CalcCost implements Implementation CalcCost interface. func (impl *ProjectionImpl) CalcCost(_ float64, children ...memo.Implementation) float64 { proj := impl.plan.(*plannercore.PhysicalProjection) - impl.cost = proj.GetCost(children[0].GetPlan().Stats().RowCount) + children[0].GetCost() + impl.cost = proj.GetCost(children[0].GetPlan().StatsInfo().RowCount) + children[0].GetCost() return impl.cost } @@ -53,7 +53,7 @@ type TiDBSelectionImpl struct { // CalcCost implements Implementation CalcCost interface. func (sel *TiDBSelectionImpl) CalcCost(_ float64, children ...memo.Implementation) float64 { - sel.cost = children[0].GetPlan().Stats().RowCount* + sel.cost = children[0].GetPlan().StatsInfo().RowCount* sel.plan.SCtx().GetSessionVars().GetCPUFactor() + children[0].GetCost() return sel.cost } @@ -70,7 +70,7 @@ type TiKVSelectionImpl struct { // CalcCost implements Implementation CalcCost interface. func (sel *TiKVSelectionImpl) CalcCost(_ float64, children ...memo.Implementation) float64 { - sel.cost = children[0].GetPlan().Stats().RowCount* + sel.cost = children[0].GetPlan().StatsInfo().RowCount* sel.plan.SCtx().GetSessionVars().GetCopCPUFactor() + children[0].GetCost() return sel.cost } @@ -88,7 +88,7 @@ type TiDBHashAggImpl struct { // CalcCost implements Implementation CalcCost interface. func (agg *TiDBHashAggImpl) CalcCost(_ float64, children ...memo.Implementation) float64 { hashAgg := agg.plan.(*plannercore.PhysicalHashAgg) - selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, true, false, 0) + selfCost := hashAgg.GetCost(children[0].GetPlan().StatsInfo().RowCount, true, false, 0) agg.cost = selfCost + children[0].GetCost() return agg.cost } @@ -113,7 +113,7 @@ type TiKVHashAggImpl struct { // CalcCost implements Implementation CalcCost interface. func (agg *TiKVHashAggImpl) CalcCost(_ float64, children ...memo.Implementation) float64 { hashAgg := agg.plan.(*plannercore.PhysicalHashAgg) - selfCost := hashAgg.GetCost(children[0].GetPlan().Stats().RowCount, false, false, 0) + selfCost := hashAgg.GetCost(children[0].GetPlan().StatsInfo().RowCount, false, false, 0) agg.cost = selfCost + children[0].GetCost() return agg.cost } @@ -143,7 +143,7 @@ type TiDBTopNImpl struct { // CalcCost implements Implementation CalcCost interface. func (impl *TiDBTopNImpl) CalcCost(_ float64, children ...memo.Implementation) float64 { topN := impl.plan.(*plannercore.PhysicalTopN) - childCount := children[0].GetPlan().Stats().RowCount + childCount := children[0].GetPlan().StatsInfo().RowCount impl.cost = topN.GetCost(childCount, true) + children[0].GetCost() return impl.cost } @@ -161,7 +161,7 @@ type TiKVTopNImpl struct { // CalcCost implements Implementation CalcCost interface. func (impl *TiKVTopNImpl) CalcCost(_ float64, children ...memo.Implementation) float64 { topN := impl.plan.(*plannercore.PhysicalTopN) - childCount := children[0].GetPlan().Stats().RowCount + childCount := children[0].GetPlan().StatsInfo().RowCount impl.cost = topN.GetCost(childCount, false) + children[0].GetCost() return impl.cost } @@ -210,8 +210,8 @@ type ApplyImpl struct { func (impl *ApplyImpl) CalcCost(_ float64, children ...memo.Implementation) float64 { apply := impl.plan.(*plannercore.PhysicalApply) impl.cost = apply.GetCost( - children[0].GetPlan().Stats().RowCount, - children[1].GetPlan().Stats().RowCount, + children[0].GetPlan().StatsInfo().RowCount, + children[1].GetPlan().StatsInfo().RowCount, children[0].GetCost(), children[1].GetCost()) return impl.cost @@ -227,7 +227,7 @@ func (impl *ApplyImpl) GetCostLimit(costLimit float64, children ...memo.Implemen // side should be (costLimit - selfCost - leftCost)/leftCount. Since // we haven't implement the rightChild, we cannot calculate the `selfCost`. // So we just use (costLimit - leftCost)/leftCount here. - leftCount, leftCost := children[0].GetPlan().Stats().RowCount, children[0].GetCost() + leftCount, leftCost := children[0].GetPlan().StatsInfo().RowCount, children[0].GetCost() apply := impl.plan.(*plannercore.PhysicalApply) if len(apply.LeftConditions) > 0 { leftCount *= plannercore.SelectionFactor diff --git a/planner/implementation/sort.go b/planner/implementation/sort.go index 582bf0726a18d..2738ba51b0789 100644 --- a/planner/implementation/sort.go +++ b/planner/implementation/sort.go @@ -33,7 +33,7 @@ func NewSortImpl(sort *plannercore.PhysicalSort) *SortImpl { // CalcCost calculates the cost of the sort Implementation. func (impl *SortImpl) CalcCost(_ float64, children ...memo.Implementation) float64 { - cnt := math.Min(children[0].GetPlan().Stats().RowCount, impl.plan.GetChildReqProps(0).ExpectedCnt) + cnt := math.Min(children[0].GetPlan().StatsInfo().RowCount, impl.plan.GetChildReqProps(0).ExpectedCnt) sort := impl.plan.(*plannercore.PhysicalSort) impl.cost = sort.GetCost(cnt, children[0].GetPlan().Schema()) + children[0].GetCost() return impl.cost