Skip to content

Commit

Permalink
expression: fully open CAST push-down switcher (#14672)
Browse files Browse the repository at this point in the history
Signed-off-by: Lonng <heng@lonng.org>
  • Loading branch information
lonng authored Feb 9, 2020
1 parent c7b43f3 commit da2ed2b
Show file tree
Hide file tree
Showing 6 changed files with 53 additions and 52 deletions.
9 changes: 9 additions & 0 deletions expression/builtin_cast.go
Original file line number Diff line number Diff line change
Expand Up @@ -865,6 +865,15 @@ func (b *builtinCastRealAsDurationSig) evalDuration(row chunk.Row) (res types.Du
return res, isNull, err
}
res, err = types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, strconv.FormatFloat(val, 'f', -1, 64), int8(b.tp.Decimal))
if err != nil {
if types.ErrTruncatedWrongVal.Equal(err) {
err = b.ctx.GetSessionVars().StmtCtx.HandleTruncate(err)
// ZeroDuration of error ErrTruncatedWrongVal needs to be considered NULL.
if res == types.ZeroDuration {
return res, true, err
}
}
}
return res, false, err
}

Expand Down
46 changes: 21 additions & 25 deletions expression/expr_to_pb.go
Original file line number Diff line number Diff line change
Expand Up @@ -373,14 +373,16 @@ func (pc PbConverter) canFuncBePushed(sf *ScalarFunction) bool {
ast.Log10,
ast.Exp,
ast.Pow,
ast.Sin,
ast.Asin,
ast.Cos,
ast.Acos,
ast.Tan,
ast.Atan,
ast.Atan2,
ast.Cot,
// Rust use the llvm math functions, which have different precision with Golang/MySQL(cmath)
// open the following switchers if we implement them in coprocessor via `cmath`
// ast.Sin,
// ast.Asin,
// ast.Cos,
// ast.Acos,
// ast.Tan,
// ast.Atan,
// ast.Atan2,
// ast.Cot,
ast.Radians,
ast.Degrees,
ast.Conv,
Expand All @@ -397,14 +399,14 @@ func (pc PbConverter) canFuncBePushed(sf *ScalarFunction) bool {
ast.BitLength,
ast.Concat,
ast.ConcatWS,
ast.Locate,
// ast.Locate,
ast.Replace,
ast.ASCII,
ast.Hex,
ast.Reverse,
ast.LTrim,
ast.RTrim,
ast.Left,
// ast.Left,
ast.Strcmp,
ast.Space,
ast.Elt,
Expand All @@ -420,22 +422,23 @@ func (pc PbConverter) canFuncBePushed(sf *ScalarFunction) bool {
ast.JSONMerge,
ast.JSONSet,
ast.JSONInsert,
ast.JSONReplace,
// ast.JSONReplace,
ast.JSONRemove,
ast.JSONLength,

// date functions.
ast.DateFormat,
ast.FromDays,
ast.ToDays,
// ast.ToDays,
ast.DayOfYear,
ast.DayOfMonth,
ast.Year,
ast.Month,
ast.Hour,
ast.Minute,
ast.Second,
ast.MicroSecond,
// FIXME: the coprocessor cannot keep the same behavior with TiDB in current compute framework
// ast.Hour,
// ast.Minute,
// ast.Second,
// ast.MicroSecond,
ast.PeriodAdd,
ast.PeriodDiff,
ast.DayName,
Expand All @@ -445,6 +448,8 @@ func (pc PbConverter) canFuncBePushed(sf *ScalarFunction) bool {
ast.SHA1,
ast.UncompressedLength,

ast.Cast,

// misc functions.
ast.InetNtoa,
ast.InetAton,
Expand All @@ -465,15 +470,6 @@ func (pc PbConverter) canFuncBePushed(sf *ScalarFunction) bool {
tipb.ScalarFuncSig_RoundDec:
return isPushdownEnabled(sf.FuncName.L)
}
case ast.Cast:
switch sf.Function.PbCode() {
case tipb.ScalarFuncSig_CastStringAsInt,
tipb.ScalarFuncSig_CastStringAsTime,
tipb.ScalarFuncSig_CastTimeAsInt:
return false
default:
return isPushdownEnabled(sf.FuncName.L)
}
case ast.Rand:
switch sf.Function.PbCode() {
case
Expand Down
8 changes: 6 additions & 2 deletions expression/expr_to_pb_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -298,10 +298,14 @@ func (s *testEvaluatorSuite) TestLikeFunc2Pb(c *C) {
likeFuncs = append(likeFuncs, fc)

pbExprs := ExpressionsToPBList(sc, likeFuncs, client)
for _, pbExpr := range pbExprs {
results := []string{
`{"tp":10000,"children":[{"tp":5,"val":"c3RyaW5n","sig":0,"field_type":{"tp":254,"flag":0,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":5,"val":"cGF0dGVybg==","sig":0,"field_type":{"tp":254,"flag":0,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":0,"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":0,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":5,"val":"JWFiYyU=","sig":0,"field_type":{"tp":254,"flag":0,"flen":-1,"decimal":-1,"collate":83,"charset":"utf8"}},{"tp":10000,"val":"CAA=","children":[{"tp":5,"val":"XA==","sig":0,"field_type":{"tp":254,"flag":0,"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"}}`,
}
for i, pbExpr := range pbExprs {
js, err := json.Marshal(pbExpr)
c.Assert(err, IsNil)
c.Assert(string(js), Equals, "null")
c.Assert(string(js), Equals, results[i])
}
}

Expand Down
8 changes: 5 additions & 3 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1215,7 +1215,9 @@ func (s *testIntegrationSuite2) TestTimeBuiltin(c *C) {
_, err = tk.Exec(`update t set a = year("aa")`)
c.Assert(terror.ErrorEqual(err, types.ErrWrongValue), IsTrue, Commentf("err %v", err))
_, err = tk.Exec(`delete from t where a = year("aa")`)
c.Assert(terror.ErrorEqual(err, types.ErrWrongValue), IsTrue, Commentf("err %v", err))
// Only `code` can be used to compare because the error `class` information
// will be lost after expression push-down
c.Assert(errors.Cause(err).(*terror.Error).Code(), Equals, types.ErrWrongValue.Code(), Commentf("err %v", err))

// for month
result = tk.MustQuery(`select month("2013-01-09"), month("2013-00-09"), month("000-01-09"), month("1-01-09"), month("20131-01-09"), month(null);`)
Expand Down Expand Up @@ -1245,7 +1247,7 @@ func (s *testIntegrationSuite2) TestTimeBuiltin(c *C) {
_, err = tk.Exec(`update t set a = month("aa")`)
c.Assert(terror.ErrorEqual(err, types.ErrWrongValue), IsTrue)
_, err = tk.Exec(`delete from t where a = month("aa")`)
c.Assert(terror.ErrorEqual(err, types.ErrWrongValue), IsTrue)
c.Assert(errors.Cause(err).(*terror.Error).Code(), Equals, types.ErrWrongValue.Code(), Commentf("err %v", err))

// for week
result = tk.MustQuery(`select week("2012-12-22"), week("2012-12-22", -2), week("2012-12-22", 0), week("2012-12-22", 1), week("2012-12-22", 2), week("2012-12-22", 200);`)
Expand Down Expand Up @@ -3134,7 +3136,7 @@ func (s *testIntegrationSuite) TestArithmeticBuiltin(c *C) {
tk.MustExec(`insert into tb5 (a) values (10);`)
e := tk.QueryToErr(`select * from tb5 where a - -9223372036854775808;`)
c.Assert(e, NotNil)
c.Assert(e.Error(), Equals, `other error: [types:1690]BIGINT value is out of range in '(Column#0 - -9223372036854775808)'`)
c.Assert(strings.HasSuffix(e.Error(), `BIGINT value is out of range in '(Column#0 - -9223372036854775808)'`), IsTrue, Commentf("err: %v", err))
tk.MustExec(`drop table tb5`)

// for multiply
Expand Down
11 changes: 3 additions & 8 deletions planner/cascades/testdata/transformation_rules_suite_out.json
Original file line number Diff line number Diff line change
Expand Up @@ -319,15 +319,10 @@
" Projection_3 input:[Group#1], test.t.b, Column#13",
"Group#1 Schema:[Column#13,test.t.b]",
" Aggregation_2 input:[Group#2], group by:plus(sin(cast(test.t.b)), sin(cast(test.t.c))), test.t.b, funcs:sum(test.t.a), firstrow(test.t.b)",
" Aggregation_7 input:[Group#3], group by:Column#16, test.t.b, funcs:sum(Column#14), firstrow(test.t.b)",
"Group#2 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]",
" TiKVSingleGather_5 input:[Group#4], table:t",
"Group#4 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]",
" TableScan_4 table:t, pk col:test.t.a",
"Group#3 Schema:[Column#14,Column#16,test.t.b]",
" TiKVSingleGather_5 input:[Group#5], table:t",
"Group#5 Schema:[Column#14,Column#16,test.t.b]",
" Aggregation_6 input:[Group#4], group by:plus(sin(cast(test.t.b)), sin(cast(test.t.c))), test.t.b, funcs:sum(test.t.a)"
" TiKVSingleGather_5 input:[Group#3], table:t",
"Group#3 Schema:[test.t.a,test.t.b,test.t.c], UniqueKey:[test.t.a]",
" TableScan_4 table:t, pk col:test.t.a"
]
}
]
Expand Down
23 changes: 9 additions & 14 deletions store/mockstore/mocktikv/cop_handler_dag.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,9 +89,6 @@ func (h *rpcHandler) handleCopDAGRequest(req *coprocessor.Request) *coprocessor.
if err == nil {
err = h.fillUpData4SelectResponse(selResp, dagReq, dagCtx, rows)
}
// FIXME: some err such as (overflow) will be include in Response.OtherError with calling this buildResp.
// Such err should only be marshal in the data but not in OtherError.
// However, we can not distinguish such err now.
return buildResp(selResp, execDetails, err)
}

Expand Down Expand Up @@ -453,10 +450,11 @@ func flagsToStatementContext(flags uint64) *stmtctx.StatementContext {
sc.TruncateAsWarning = (flags & model.FlagTruncateAsWarning) > 0
sc.InInsertStmt = (flags & model.FlagInInsertStmt) > 0
sc.InSelectStmt = (flags & model.FlagInSelectStmt) > 0
sc.InDeleteStmt = (flags & model.FlagInUpdateOrDeleteStmt) > 0
sc.OverflowAsWarning = (flags & model.FlagOverflowAsWarning) > 0
sc.IgnoreZeroInDate = (flags & model.FlagIgnoreZeroInDate) > 0
sc.DividedByZeroAsWarning = (flags & model.FlagDividedByZeroAsWarning) > 0
// TODO set FlagInUpdateOrDeleteStmt, FlagInUnionStmt,
// TODO set FlagInUnionStmt,
return sc
}

Expand Down Expand Up @@ -719,16 +717,13 @@ func buildResp(selResp *tipb.SelectResponse, execDetails []*execDetail, err erro
selResp.ExecutionSummaries = execSummary
}

if err != nil {
if locked, ok := errors.Cause(err).(*ErrLocked); ok {
resp.Locked = &kvrpcpb.LockInfo{
Key: locked.Key,
PrimaryLock: locked.Primary,
LockVersion: locked.StartTS,
LockTtl: locked.TTL,
}
} else {
resp.OtherError = err.Error()
// Select errors have been contained in `SelectResponse.Error`
if locked, ok := errors.Cause(err).(*ErrLocked); ok {
resp.Locked = &kvrpcpb.LockInfo{
Key: locked.Key,
PrimaryLock: locked.Primary,
LockVersion: locked.StartTS,
LockTtl: locked.TTL,
}
}
data, err := proto.Marshal(selResp)
Expand Down

0 comments on commit da2ed2b

Please sign in to comment.