Skip to content

Commit

Permalink
expression: replace EvalWithInnerCtx with Eval for PbConverter
Browse files Browse the repository at this point in the history
…and `ExplainInfo` (#48917)

close #48914
  • Loading branch information
lcwangchao authored Nov 27, 2023
1 parent 5610645 commit ccbd1b5
Show file tree
Hide file tree
Showing 26 changed files with 292 additions and 284 deletions.
2 changes: 1 addition & 1 deletion pkg/executor/distsql.go
Original file line number Diff line number Diff line change
Expand Up @@ -339,7 +339,7 @@ func (e *IndexReaderExecutor) open(ctx context.Context, kvRanges []kv.KeyRange)
if err != nil {
return err
}
pbConditions, err := expression.ExpressionsToPBList(e.Ctx().GetSessionVars().StmtCtx, []expression.Expression{inCondition}, e.Ctx().GetClient())
pbConditions, err := expression.ExpressionsToPBList(e.Ctx(), []expression.Expression{inCondition}, e.Ctx().GetClient())
if err != nil {
return err
}
Expand Down
5 changes: 2 additions & 3 deletions pkg/expression/aggregation/agg_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -101,7 +101,7 @@ func (desc *baseFuncDesc) GetTiPBExpr(tryWindowDesc bool) (tp tipb.ExprType) {

// AggFuncToPBExpr converts aggregate function to pb.
func AggFuncToPBExpr(sctx sessionctx.Context, client kv.Client, aggFunc *AggFuncDesc, storeType kv.StoreType) (*tipb.Expr, error) {
pc := expression.NewPBConverter(client, sctx.GetSessionVars().StmtCtx)
pc := expression.NewPBConverter(client, sctx)
tp := aggFunc.GetTiPBExpr(false)
if !client.IsRequestTypeSupported(kv.ReqTypeSelect, int64(tp)) {
return nil, errors.New("select request is not supported by client")
Expand All @@ -122,9 +122,8 @@ func AggFuncToPBExpr(sctx sessionctx.Context, client kv.Client, aggFunc *AggFunc

if tp == tipb.ExprType_GroupConcat {
orderBy := make([]*tipb.ByItem, 0, len(aggFunc.OrderByItems))
sc := sctx.GetSessionVars().StmtCtx
for _, arg := range aggFunc.OrderByItems {
pbArg := expression.SortByItemToPB(sc, client, arg.Expr, arg.Desc)
pbArg := expression.SortByItemToPB(sctx, client, arg.Expr, arg.Desc)
if pbArg == nil {
return nil, errors.New(aggFunc.String() + " can't be converted to PB.")
}
Expand Down
9 changes: 5 additions & 4 deletions pkg/expression/aggregation/explain.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,10 +19,11 @@ import (
"fmt"

"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/sessionctx"
)

// ExplainAggFunc generates explain information for a aggregation function.
func ExplainAggFunc(agg *AggFuncDesc, normalized bool) string {
func ExplainAggFunc(ctx sessionctx.Context, agg *AggFuncDesc, normalized bool) string {
var buffer bytes.Buffer
fmt.Fprintf(&buffer, "%s(", agg.Name)
if agg.HasDistinct {
Expand All @@ -37,13 +38,13 @@ func ExplainAggFunc(agg *AggFuncDesc, normalized bool) string {
if normalized {
fmt.Fprintf(&buffer, "%s desc", item.Expr.ExplainNormalizedInfo())
} else {
fmt.Fprintf(&buffer, "%s desc", item.Expr.ExplainInfo())
fmt.Fprintf(&buffer, "%s desc", item.Expr.ExplainInfo(ctx))
}
} else {
if normalized {
fmt.Fprintf(&buffer, "%s", item.Expr.ExplainNormalizedInfo())
} else {
fmt.Fprintf(&buffer, "%s", item.Expr.ExplainInfo())
fmt.Fprintf(&buffer, "%s", item.Expr.ExplainInfo(ctx))
}
}

Expand All @@ -59,7 +60,7 @@ func ExplainAggFunc(agg *AggFuncDesc, normalized bool) string {
if normalized {
buffer.WriteString(arg.ExplainNormalizedInfo())
} else {
buffer.WriteString(arg.ExplainInfo())
buffer.WriteString(arg.ExplainInfo(ctx))
}
}
buffer.WriteString(")")
Expand Down
4 changes: 2 additions & 2 deletions pkg/expression/aggregation/window_func.go
Original file line number Diff line number Diff line change
Expand Up @@ -125,7 +125,7 @@ func (s *WindowFuncDesc) Clone() *WindowFuncDesc {

// WindowFuncToPBExpr converts aggregate function to pb.
func WindowFuncToPBExpr(sctx sessionctx.Context, client kv.Client, desc *WindowFuncDesc) *tipb.Expr {
pc := expression.NewPBConverter(client, sctx.GetSessionVars().StmtCtx)
pc := expression.NewPBConverter(client, sctx)
tp := desc.GetTiPBExpr(true)
if !client.IsRequestTypeSupported(kv.ReqTypeSelect, int64(tp)) {
return nil
Expand All @@ -145,7 +145,7 @@ func WindowFuncToPBExpr(sctx sessionctx.Context, client kv.Client, desc *WindowF
// CanPushDownToTiFlash control whether a window function desc can be push down to tiflash.
func (s *WindowFuncDesc) CanPushDownToTiFlash(ctx sessionctx.Context) bool {
// args
if !expression.CanExprsPushDown(ctx.GetSessionVars().StmtCtx, s.Args, ctx.GetClient(), kv.TiFlash) {
if !expression.CanExprsPushDown(ctx, s.Args, ctx.GetClient(), kv.TiFlash) {
return false
}
// window functions
Expand Down
8 changes: 4 additions & 4 deletions pkg/expression/constant_fold.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,7 +58,7 @@ func isNullHandler(ctx sessionctx.Context, expr *ScalarFunction) (Expression, bo
// Failed to fold this expr to a constant, print the DEBUG log and
// return the original expression to let the error to be evaluated
// again, in that time, the error is returned to the client.
logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", expr.ExplainInfo()), zap.Error(err))
logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", expr.ExplainInfo(ctx)), zap.Error(err))
return expr, isDeferredConst
}
if isDeferredConst {
Expand All @@ -81,7 +81,7 @@ func ifFoldHandler(ctx sessionctx.Context, expr *ScalarFunction) (Expression, bo
// Failed to fold this expr to a constant, print the DEBUG log and
// return the original expression to let the error to be evaluated
// again, in that time, the error is returned to the client.
logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", expr.ExplainInfo()), zap.Error(err))
logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", expr.ExplainInfo(ctx)), zap.Error(err))
return expr, false
}
if !isNull0 && arg0 != 0 {
Expand Down Expand Up @@ -230,7 +230,7 @@ func foldConstant(ctx sessionctx.Context, expr Expression) (Expression, bool) {
}
}
if err != nil {
logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", x.ExplainInfo()), zap.Error(err))
logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", x.ExplainInfo(ctx)), zap.Error(err))
return expr, isDeferredConst
}
if isDeferredConst {
Expand All @@ -248,7 +248,7 @@ func foldConstant(ctx sessionctx.Context, expr Expression) (Expression, bool) {
} else if x.DeferredExpr != nil {
value, err := x.DeferredExpr.Eval(ctx, chunk.Row{})
if err != nil {
logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", x.ExplainInfo()), zap.Error(err))
logutil.BgLogger().Debug("fold expression to constant", zap.String("expression", x.ExplainInfo(ctx)), zap.Error(err))
return expr, true
}
return &Constant{Value: value, RetType: x.RetType, DeferredExpr: x.DeferredExpr}, true
Expand Down
67 changes: 40 additions & 27 deletions pkg/expression/explain.go
Original file line number Diff line number Diff line change
Expand Up @@ -21,16 +21,20 @@ import (
"strings"

"github.com/pingcap/tidb/pkg/parser/ast"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/intest"
)

// ExplainInfo implements the Expression interface.
func (expr *ScalarFunction) ExplainInfo() string {
return expr.explainInfo(false)
func (expr *ScalarFunction) ExplainInfo(ctx sessionctx.Context) string {
return expr.explainInfo(ctx, false)
}

func (expr *ScalarFunction) explainInfo(normalized bool) string {
func (expr *ScalarFunction) explainInfo(ctx sessionctx.Context, normalized bool) string {
// we only need ctx for non-normalized explain info.
intest.Assert(normalized || ctx != nil)
var buffer bytes.Buffer
fmt.Fprintf(&buffer, "%s(", expr.FuncName.L)
switch expr.FuncName.L {
Expand All @@ -39,7 +43,8 @@ func (expr *ScalarFunction) explainInfo(normalized bool) string {
if normalized {
buffer.WriteString(arg.ExplainNormalizedInfo())
} else {
buffer.WriteString(arg.ExplainInfo())
intest.Assert(ctx != nil)
buffer.WriteString(arg.ExplainInfo(ctx))
}
buffer.WriteString(", ")
buffer.WriteString(expr.RetType.String())
Expand All @@ -49,7 +54,8 @@ func (expr *ScalarFunction) explainInfo(normalized bool) string {
if normalized {
buffer.WriteString(arg.ExplainNormalizedInfo())
} else {
buffer.WriteString(arg.ExplainInfo())
intest.Assert(ctx != nil)
buffer.WriteString(arg.ExplainInfo(ctx))
}
if i+1 < len(expr.GetArgs()) {
buffer.WriteString(", ")
Expand All @@ -62,7 +68,7 @@ func (expr *ScalarFunction) explainInfo(normalized bool) string {

// ExplainNormalizedInfo implements the Expression interface.
func (expr *ScalarFunction) ExplainNormalizedInfo() string {
return expr.explainInfo(true)
return expr.explainInfo(nil, true)
}

// ExplainNormalizedInfo4InList implements the Expression interface.
Expand Down Expand Up @@ -90,30 +96,35 @@ func (expr *ScalarFunction) ExplainNormalizedInfo4InList() string {
return buffer.String()
}

// ExplainInfo implements the Expression interface.
func (col *Column) ExplainInfo() string {
// ColumnExplainInfo returns the explained info for column.
func (col *Column) ColumnExplainInfo(normalized bool) string {
if normalized {
if col.OrigName != "" {
return col.OrigName
}
return "?"
}
return col.String()
}

// ExplainInfo implements the Expression interface.
func (col *Column) ExplainInfo(sessionctx.Context) string {
return col.ColumnExplainInfo(false)
}

// ExplainNormalizedInfo implements the Expression interface.
func (col *Column) ExplainNormalizedInfo() string {
if col.OrigName != "" {
return col.OrigName
}
return "?"
return col.ColumnExplainInfo(true)
}

// ExplainNormalizedInfo4InList implements the Expression interface.
func (col *Column) ExplainNormalizedInfo4InList() string {
if col.OrigName != "" {
return col.OrigName
}
return "?"
return col.ColumnExplainInfo(true)
}

// ExplainInfo implements the Expression interface.
func (expr *Constant) ExplainInfo() string {
dt, err := expr.EvalWithInnerCtx(chunk.Row{})
func (expr *Constant) ExplainInfo(ctx sessionctx.Context) string {
dt, err := expr.Eval(ctx, chunk.Row{})
if err != nil {
return "not recognized const value"
}
Expand Down Expand Up @@ -179,16 +190,17 @@ func ExplainExpressionList(exprs []Expression, schema *Schema) string {
// SortedExplainExpressionList generates explain information for a list of expressions in order.
// In some scenarios, the expr's order may not be stable when executing multiple times.
// So we add a sort to make its explain result stable.
func SortedExplainExpressionList(exprs []Expression) []byte {
return sortedExplainExpressionList(exprs, false, false)
func SortedExplainExpressionList(ctx sessionctx.Context, exprs []Expression) []byte {
return sortedExplainExpressionList(ctx, exprs, false, false)
}

// SortedExplainExpressionListIgnoreInlist generates explain information for a list of expressions in order.
func SortedExplainExpressionListIgnoreInlist(exprs []Expression) []byte {
return sortedExplainExpressionList(exprs, false, true)
return sortedExplainExpressionList(nil, exprs, false, true)
}

func sortedExplainExpressionList(exprs []Expression, normalized bool, ignoreInlist bool) []byte {
func sortedExplainExpressionList(ctx sessionctx.Context, exprs []Expression, normalized bool, ignoreInlist bool) []byte {
intest.Assert(ignoreInlist || normalized || ctx != nil)
buffer := bytes.NewBufferString("")
exprInfos := make([]string, 0, len(exprs))
for _, expr := range exprs {
Expand All @@ -197,7 +209,8 @@ func sortedExplainExpressionList(exprs []Expression, normalized bool, ignoreInli
} else if normalized {
exprInfos = append(exprInfos, expr.ExplainNormalizedInfo())
} else {
exprInfos = append(exprInfos, expr.ExplainInfo())
intest.Assert(ctx != nil)
exprInfos = append(exprInfos, expr.ExplainInfo(ctx))
}
}
slices.Sort(exprInfos)
Expand All @@ -212,7 +225,7 @@ func sortedExplainExpressionList(exprs []Expression, normalized bool, ignoreInli

// SortedExplainNormalizedExpressionList is same like SortedExplainExpressionList, but use for generating normalized information.
func SortedExplainNormalizedExpressionList(exprs []Expression) []byte {
return sortedExplainExpressionList(exprs, true, false)
return sortedExplainExpressionList(nil, exprs, true, false)
}

// SortedExplainNormalizedScalarFuncList is same like SortedExplainExpressionList, but use for generating normalized information.
Expand All @@ -221,14 +234,14 @@ func SortedExplainNormalizedScalarFuncList(exprs []*ScalarFunction) []byte {
for i := range exprs {
expressions[i] = exprs[i]
}
return sortedExplainExpressionList(expressions, true, false)
return sortedExplainExpressionList(nil, expressions, true, false)
}

// ExplainColumnList generates explain information for a list of columns.
func ExplainColumnList(cols []*Column) []byte {
func ExplainColumnList(ctx sessionctx.Context, cols []*Column) []byte {
buffer := bytes.NewBufferString("")
for i, col := range cols {
buffer.WriteString(col.ExplainInfo())
buffer.WriteString(col.ExplainInfo(ctx))
if i+1 < len(cols) {
buffer.WriteString(", ")
}
Expand Down
29 changes: 15 additions & 14 deletions pkg/expression/expr_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,7 @@ import (
"github.com/pingcap/tidb/pkg/kv"
"github.com/pingcap/tidb/pkg/parser/mysql"
ast "github.com/pingcap/tidb/pkg/parser/types"
"github.com/pingcap/tidb/pkg/sessionctx/stmtctx"
"github.com/pingcap/tidb/pkg/sessionctx"
"github.com/pingcap/tidb/pkg/types"
"github.com/pingcap/tidb/pkg/util/chunk"
"github.com/pingcap/tidb/pkg/util/codec"
Expand All @@ -34,8 +34,8 @@ import (
)

// ExpressionsToPBList converts expressions to tipb.Expr list for new plan.
func ExpressionsToPBList(sc *stmtctx.StatementContext, exprs []Expression, client kv.Client) (pbExpr []*tipb.Expr, err error) {
pc := PbConverter{client: client, sc: sc}
func ExpressionsToPBList(ctx sessionctx.Context, exprs []Expression, client kv.Client) (pbExpr []*tipb.Expr, err error) {
pc := PbConverter{client: client, ctx: ctx}
for _, expr := range exprs {
v := pc.ExprToPB(expr)
if v == nil {
Expand All @@ -49,12 +49,12 @@ func ExpressionsToPBList(sc *stmtctx.StatementContext, exprs []Expression, clien
// PbConverter supplies methods to convert TiDB expressions to TiPB.
type PbConverter struct {
client kv.Client
sc *stmtctx.StatementContext
ctx sessionctx.Context
}

// NewPBConverter creates a PbConverter.
func NewPBConverter(client kv.Client, sc *stmtctx.StatementContext) PbConverter {
return PbConverter{client: client, sc: sc}
func NewPBConverter(client kv.Client, ctx sessionctx.Context) PbConverter {
return PbConverter{client: client, ctx: ctx}
}

// ExprToPB converts Expression to TiPB.
Expand All @@ -78,9 +78,9 @@ func (pc PbConverter) ExprToPB(expr Expression) *tipb.Expr {

func (pc PbConverter) conOrCorColToPBExpr(expr Expression) *tipb.Expr {
ft := expr.GetType()
d, err := expr.EvalWithInnerCtx(chunk.Row{})
d, err := expr.Eval(pc.ctx, chunk.Row{})
if err != nil {
logutil.BgLogger().Error("eval constant or correlated column", zap.String("expression", expr.ExplainInfo()), zap.Error(err))
logutil.BgLogger().Error("eval constant or correlated column", zap.String("expression", expr.ExplainInfo(pc.ctx)), zap.Error(err))
return nil
}
tp, val, ok := pc.encodeDatum(ft, d)
Expand Down Expand Up @@ -143,8 +143,9 @@ func (pc *PbConverter) encodeDatum(ft *types.FieldType, d types.Datum) (tipb.Exp
case types.KindMysqlTime:
if pc.client.IsRequestTypeSupported(kv.ReqTypeDAG, int64(tipb.ExprType_MysqlTime)) {
tp = tipb.ExprType_MysqlTime
val, err := codec.EncodeMySQLTime(pc.sc.TimeZone(), d.GetMysqlTime(), ft.GetType(), nil)
err = pc.sc.HandleError(err)
sc := pc.ctx.GetSessionVars().StmtCtx
val, err := codec.EncodeMySQLTime(sc.TimeZone(), d.GetMysqlTime(), ft.GetType(), nil)
err = sc.HandleError(err)
if err != nil {
logutil.BgLogger().Error("encode mysql time", zap.Error(err))
return tp, nil, false
Expand Down Expand Up @@ -277,8 +278,8 @@ func (pc PbConverter) scalarFuncToPBExpr(expr *ScalarFunction) *tipb.Expr {
}

// GroupByItemToPB converts group by items to pb.
func GroupByItemToPB(sc *stmtctx.StatementContext, client kv.Client, expr Expression) *tipb.ByItem {
pc := PbConverter{client: client, sc: sc}
func GroupByItemToPB(ctx sessionctx.Context, client kv.Client, expr Expression) *tipb.ByItem {
pc := PbConverter{client: client, ctx: ctx}
e := pc.ExprToPB(expr)
if e == nil {
return nil
Expand All @@ -287,8 +288,8 @@ func GroupByItemToPB(sc *stmtctx.StatementContext, client kv.Client, expr Expres
}

// SortByItemToPB converts order by items to pb.
func SortByItemToPB(sc *stmtctx.StatementContext, client kv.Client, expr Expression, desc bool) *tipb.ByItem {
pc := PbConverter{client: client, sc: sc}
func SortByItemToPB(ctx sessionctx.Context, client kv.Client, expr Expression, desc bool) *tipb.ByItem {
pc := PbConverter{client: client, ctx: ctx}
e := pc.ExprToPB(expr)
if e == nil {
return nil
Expand Down
Loading

0 comments on commit ccbd1b5

Please sign in to comment.