Skip to content

Commit

Permalink
expression: return an error when finding some expressions cannot be p…
Browse files Browse the repository at this point in the history
…ushed down to avoid panic (#16671) (#16869)
  • Loading branch information
sre-bot authored Apr 27, 2020
1 parent 3b81a7f commit 506224a
Show file tree
Hide file tree
Showing 3 changed files with 73 additions and 36 deletions.
7 changes: 6 additions & 1 deletion expression/expr_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -18,6 +18,7 @@ import (
"github.com/pingcap/errors"
"github.com/pingcap/failpoint"
"github.com/pingcap/parser/mysql"
"github.com/pingcap/parser/terror"
"github.com/pingcap/tidb/kv"
"github.com/pingcap/tidb/sessionctx/stmtctx"
"github.com/pingcap/tidb/types"
Expand All @@ -30,10 +31,14 @@ import (
)

// ExpressionsToPBList converts expressions to tipb.Expr list for new plan.
func ExpressionsToPBList(sc *stmtctx.StatementContext, exprs []Expression, client kv.Client) (pbExpr []*tipb.Expr) {
func ExpressionsToPBList(sc *stmtctx.StatementContext, exprs []Expression, client kv.Client) (pbExpr []*tipb.Expr, err error) {
pc := PbConverter{client: client, sc: sc}
for _, expr := range exprs {
v := pc.ExprToPB(expr)
if v == nil {
return nil, terror.ClassOptimizer.New(mysql.ErrInternal, mysql.MySQLErrName[mysql.ErrInternal]).
GenWithStack("expression %v cannot be pushed down", expr)
}
pbExpr = append(pbExpr, v)
}
return
Expand Down
84 changes: 52 additions & 32 deletions expression/expr_to_pb_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -114,7 +114,8 @@ func (s *testEvaluatorSuite) TestConstant2Pb(c *C) {
c.Assert(len(pushed), Equals, len(constExprs)-3)
c.Assert(len(remained), Equals, 3)

pbExprs := ExpressionsToPBList(sc, constExprs, client)
pbExprs, err := ExpressionsToPBList(sc, constExprs, client)
c.Assert(err, IsNil)
jsons := []string{
"{\"tp\":0,\"sig\":0}",
"{\"tp\":1,\"val\":\"gAAAAAAAAGQ=\",\"sig\":0}",
Expand Down Expand Up @@ -152,9 +153,9 @@ func (s *testEvaluatorSuite) TestColumn2Pb(c *C) {
c.Assert(len(pushed), Equals, 0)
c.Assert(len(remained), Equals, len(colExprs))

pbExprs := ExpressionsToPBList(sc, colExprs, client)
for _, pbExpr := range pbExprs {
c.Assert(pbExpr, IsNil)
for _, col := range colExprs { // cannot be pushed down
_, err := ExpressionsToPBList(sc, []Expression{col}, client)
c.Assert(err, NotNil)
}

colExprs = colExprs[:0]
Expand Down Expand Up @@ -184,7 +185,8 @@ func (s *testEvaluatorSuite) TestColumn2Pb(c *C) {
c.Assert(len(pushed), Equals, len(colExprs))
c.Assert(len(remained), Equals, 0)

pbExprs = ExpressionsToPBList(sc, colExprs, client)
pbExprs, err := ExpressionsToPBList(sc, colExprs, client)
c.Assert(err, IsNil)
jsons := []string{
"{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}",
"{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":2,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}",
Expand Down Expand Up @@ -243,7 +245,8 @@ func (s *testEvaluatorSuite) TestCompareFunc2Pb(c *C) {
c.Assert(len(pushed), Equals, len(compareExprs))
c.Assert(len(remained), Equals, 0)

pbExprs := ExpressionsToPBList(sc, compareExprs, client)
pbExprs, err := ExpressionsToPBList(sc, compareExprs, client)
c.Assert(err, IsNil)
c.Assert(len(pbExprs), Equals, len(compareExprs))
jsons := []string{
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":8,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":100,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
Expand Down Expand Up @@ -286,7 +289,8 @@ func (s *testEvaluatorSuite) TestLikeFunc2Pb(c *C) {
c.Assert(err, IsNil)
likeFuncs = append(likeFuncs, fc)

pbExprs := ExpressionsToPBList(sc, likeFuncs, client)
pbExprs, err := ExpressionsToPBList(sc, likeFuncs, client)
c.Assert(err, IsNil)
results := []string{
`{"tp":10000,"children":[{"tp":5,"val":"c3RyaW5n","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":5,"val":"cGF0dGVybg==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}}],"sig":30,"field_type":{"tp":8,"flag":128,"flen":-1,"decimal":0,"collate":63,"charset":"binary"}}],"sig":4310,"field_type":{"tp":8,"flag":128,"flen":1,"decimal":0,"collate":63,"charset":"binary"}}`,
`{"tp":10000,"children":[{"tp":5,"val":"c3RyaW5n","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":5,"val":"JWFiYyU=","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":1,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}}],"sig":30,"field_type":{"tp":8,"flag":128,"flen":-1,"decimal":0,"collate":63,"charset":"binary"}}],"sig":4310,"field_type":{"tp":8,"flag":128,"flen":1,"decimal":0,"collate":63,"charset":"binary"}}`,
Expand All @@ -304,7 +308,7 @@ func (s *testEvaluatorSuite) TestArithmeticalFunc2Pb(c *C) {
client := new(mock.Client)
dg := new(dataGen4Expr2PbTest)

funcNames := []string{ast.Plus, ast.Minus, ast.Mul, ast.Div, ast.Mod, ast.IntDiv}
funcNames := []string{ast.Plus, ast.Minus, ast.Mul, ast.Div}
for _, funcName := range funcNames {
fc, err := NewFunction(
mock.NewContext(),
Expand All @@ -322,17 +326,26 @@ func (s *testEvaluatorSuite) TestArithmeticalFunc2Pb(c *C) {
jsons[ast.Mul] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":208,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"}}"
jsons[ast.Div] = "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":5,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":211,\"field_type\":{\"tp\":5,\"flag\":128,\"flen\":23,\"decimal\":-1,\"collate\":63,\"charset\":\"binary\"}}"

pbExprs := ExpressionsToPBList(sc, arithmeticalFuncs, client)
pbExprs, err := ExpressionsToPBList(sc, arithmeticalFuncs, client)
c.Assert(err, IsNil)
for i, pbExpr := range pbExprs {
switch funcNames[i] {
case ast.Mod, ast.IntDiv:
c.Assert(pbExpr, IsNil, Commentf("%v\n", funcNames[i]))
default:
c.Assert(pbExpr, NotNil)
js, err := json.Marshal(pbExpr)
c.Assert(err, IsNil)
c.Assert(string(js), Equals, jsons[funcNames[i]], Commentf("%v\n", funcNames[i]))
}
c.Assert(pbExpr, NotNil)
js, err := json.Marshal(pbExpr)
c.Assert(err, IsNil)
c.Assert(string(js), Equals, jsons[funcNames[i]], Commentf("%v\n", funcNames[i]))
}

funcNames = []string{ast.Mod, ast.IntDiv} // cannot be pushed down
for _, funcName := range funcNames {
fc, err := NewFunction(
mock.NewContext(),
funcName,
types.NewFieldType(mysql.TypeUnspecified),
dg.genColumn(mysql.TypeDouble, 1),
dg.genColumn(mysql.TypeDouble, 2))
c.Assert(err, IsNil)
_, err = ExpressionsToPBList(sc, []Expression{fc}, client)
c.Assert(err, NotNil)
}
}

Expand All @@ -348,7 +361,8 @@ func (s *testEvaluatorSuite) TestDateFunc2Pb(c *C) {
dg.genColumn(mysql.TypeString, 2))
c.Assert(err, IsNil)
funcs := []Expression{fc}
pbExprs := ExpressionsToPBList(sc, funcs, client)
pbExprs, err := ExpressionsToPBList(sc, funcs, client)
c.Assert(err, IsNil)
c.Assert(pbExprs[0], NotNil)
js, err := json.Marshal(pbExprs[0])
c.Assert(err, IsNil)
Expand Down Expand Up @@ -377,7 +391,8 @@ func (s *testEvaluatorSuite) TestLogicalFunc2Pb(c *C) {
logicalFuncs = append(logicalFuncs, fc)
}

pbExprs := ExpressionsToPBList(sc, logicalFuncs, client)
pbExprs, err := ExpressionsToPBList(sc, logicalFuncs, client)
c.Assert(err, IsNil)
jsons := []string{
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3101,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":1,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3102,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
Expand Down Expand Up @@ -413,7 +428,8 @@ func (s *testEvaluatorSuite) TestBitwiseFunc2Pb(c *C) {
bitwiseFuncs = append(bitwiseFuncs, fc)
}

pbExprs := ExpressionsToPBList(sc, bitwiseFuncs, client)
pbExprs, err := ExpressionsToPBList(sc, bitwiseFuncs, client)
c.Assert(err, IsNil)
jsons := []string{
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3118,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3119,\"field_type\":{\"tp\":8,\"flag\":160,\"flen\":20,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
Expand Down Expand Up @@ -484,26 +500,26 @@ func (s *testEvaluatorSerialSuites) TestPushDownSwitcher(c *C) {
c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/PushDownTestSwitcher", `return("all")`), IsNil)
defer func() { c.Assert(failpoint.Disable("github.com/pingcap/tidb/expression/PushDownTestSwitcher"), IsNil) }()

pbExprs := ExpressionsToPBList(sc, funcs, client)
pbExprs, err := ExpressionsToPBList(sc, funcs, client)
c.Assert(err, IsNil)
c.Assert(len(pbExprs), Equals, len(cases))
for i, pbExpr := range pbExprs {
c.Assert(pbExpr.Sig, Equals, cases[i].sig, Commentf("function: %s, sig: %v", cases[i].name, cases[i].sig))
}

// All disabled
c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/PushDownTestSwitcher", `return("")`), IsNil)
pbExprs = ExpressionsToPBList(sc, funcs, client)
c.Assert(len(pbExprs), Equals, len(cases))
for i, pbExpr := range pbExprs {
pc := PbConverter{client: client, sc: sc}
for i := range funcs {
pbExpr := pc.ExprToPB(funcs[i])
c.Assert(pbExpr, IsNil, Commentf("function: %s, sig: %v", cases[i].name, cases[i].sig))
}

// Partial enabled
fpexpr := fmt.Sprintf(`return("%s")`, strings.Join(enabled, ","))
c.Assert(failpoint.Enable("github.com/pingcap/tidb/expression/PushDownTestSwitcher", fpexpr), IsNil)
pbExprs = ExpressionsToPBList(sc, funcs, client)
c.Assert(len(pbExprs), Equals, len(cases))
for i, pbExpr := range pbExprs {
for i := range funcs {
pbExpr := pc.ExprToPB(funcs[i])
if !cases[i].enable {
c.Assert(pbExpr, IsNil, Commentf("function: %s, sig: %v", cases[i].name, cases[i].sig))
continue
Expand Down Expand Up @@ -539,7 +555,8 @@ func (s *testEvaluatorSuite) TestControlFunc2Pb(c *C) {
controlFuncs = append(controlFuncs, fc)
}

pbExprs := ExpressionsToPBList(sc, controlFuncs, client)
pbExprs, err := ExpressionsToPBList(sc, controlFuncs, client)
c.Assert(err, IsNil)
jsons := []string{
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4208,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"\"}}",
"{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}},{\"tp\":201,\"val\":\"gAAAAAAAAAM=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4107,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":-1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
Expand Down Expand Up @@ -571,7 +588,8 @@ func (s *testEvaluatorSuite) TestOtherFunc2Pb(c *C) {
otherFuncs = append(otherFuncs, fc)
}

pbExprs := ExpressionsToPBList(sc, otherFuncs, client)
pbExprs, err := ExpressionsToPBList(sc, otherFuncs, client)
c.Assert(err, IsNil)
jsons := map[string]string{
ast.Coalesce: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":4201,\"field_type\":{\"tp\":3,\"flag\":128,\"flen\":0,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}",
ast.IsNull: "{\"tp\":10000,\"children\":[{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":3,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":63,\"charset\":\"\"}}],\"sig\":3116,\"field_type\":{\"tp\":8,\"flag\":128,\"flen\":1,\"decimal\":0,\"collate\":63,\"charset\":\"binary\"}}",
Expand Down Expand Up @@ -738,7 +756,8 @@ func (s *testEvaluatorSerialSuites) TestNewCollationsEnabled(c *C) {
colExprs = append(colExprs, columnCollation(dg.genColumn(mysql.TypeVarchar, 5), "utf8_bin"))
pushed, _ := PushDownExprs(sc, colExprs, client, kv.UnSpecified)
c.Assert(len(pushed), Equals, len(colExprs))
pbExprs := ExpressionsToPBList(sc, colExprs, client)
pbExprs, err := ExpressionsToPBList(sc, colExprs, client)
c.Assert(err, IsNil)
jsons := []string{
"{\"tp\":201,\"val\":\"gAAAAAAAAAE=\",\"sig\":0,\"field_type\":{\"tp\":15,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-46,\"charset\":\"\"}}",
"{\"tp\":201,\"val\":\"gAAAAAAAAAI=\",\"sig\":0,\"field_type\":{\"tp\":15,\"flag\":0,\"flen\":-1,\"decimal\":-1,\"collate\":-46,\"charset\":\"\"}}",
Expand Down Expand Up @@ -773,7 +792,8 @@ func (s *testEvalSerialSuite) TestPushCollationDown(c *C) {
tps := []*types.FieldType{types.NewFieldType(mysql.TypeVarchar), types.NewFieldType(mysql.TypeVarchar)}
for _, coll := range []string{charset.CollationBin, charset.CollationLatin1, charset.CollationUTF8, charset.CollationUTF8MB4} {
fc.SetCharsetAndCollation("binary", coll, types.UnspecifiedLength) // only collation matters
pbExpr := ExpressionsToPBList(sc, []Expression{fc}, client)
pbExpr, err := ExpressionsToPBList(sc, []Expression{fc}, client)
c.Assert(err, IsNil)
expr, err := PBToExpr(pbExpr[0], tps, sc)
c.Assert(err, IsNil)
_, eColl, _ := expr.CharsetAndCollation(nil)
Expand Down
18 changes: 15 additions & 3 deletions planner/core/plan_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -36,8 +36,12 @@ func (p *basePhysicalPlan) ToPB(_ sessionctx.Context) (*tipb.Executor, error) {
func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) {
sc := ctx.GetSessionVars().StmtCtx
client := ctx.GetClient()
groupByExprs, err := expression.ExpressionsToPBList(sc, p.GroupByItems, client)
if err != nil {
return nil, err
}
aggExec := &tipb.Aggregation{
GroupBy: expression.ExpressionsToPBList(sc, p.GroupByItems, client),
GroupBy: groupByExprs,
}
for _, aggFunc := range p.AggFuncs {
aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc))
Expand All @@ -49,8 +53,12 @@ func (p *PhysicalHashAgg) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) {
func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) {
sc := ctx.GetSessionVars().StmtCtx
client := ctx.GetClient()
groupByExprs, err := expression.ExpressionsToPBList(sc, p.GroupByItems, client)
if err != nil {
return nil, err
}
aggExec := &tipb.Aggregation{
GroupBy: expression.ExpressionsToPBList(sc, p.GroupByItems, client),
GroupBy: groupByExprs,
}
for _, aggFunc := range p.AggFuncs {
aggExec.AggFunc = append(aggExec.AggFunc, aggregation.AggFuncToPBExpr(sc, client, aggFunc))
Expand All @@ -62,8 +70,12 @@ func (p *PhysicalStreamAgg) ToPB(ctx sessionctx.Context) (*tipb.Executor, error)
func (p *PhysicalSelection) ToPB(ctx sessionctx.Context) (*tipb.Executor, error) {
sc := ctx.GetSessionVars().StmtCtx
client := ctx.GetClient()
conditions, err := expression.ExpressionsToPBList(sc, p.Conditions, client)
if err != nil {
return nil, err
}
selExec := &tipb.Selection{
Conditions: expression.ExpressionsToPBList(sc, p.Conditions, client),
Conditions: conditions,
}
return &tipb.Executor{Tp: tipb.ExecType_TypeSelection, Selection: selExec}, nil
}
Expand Down

0 comments on commit 506224a

Please sign in to comment.