Skip to content

Commit

Permalink
expression,planner: support non-deterministic functions (e.g., now) i…
Browse files Browse the repository at this point in the history
…n the plan cache (#8105)
  • Loading branch information
dbjoa authored and ngaut committed Nov 1, 2018
1 parent b7f431e commit 5374ff8
Show file tree
Hide file tree
Showing 8 changed files with 236 additions and 55 deletions.
2 changes: 2 additions & 0 deletions executor/executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -1173,6 +1173,8 @@ func ResetContextOfStmt(ctx sessionctx.Context, s ast.StmtNode) (err error) {
sc := new(stmtctx.StatementContext)
sc.TimeZone = vars.Location()
sc.MemTracker = memory.NewTracker(s.Text(), vars.MemQuotaQuery)
sc.NowTs = time.Time{}
sc.SysTs = time.Time{}
switch config.GetGlobalConfig().OOMAction {
case config.OOMActionCancel:
sc.MemTracker.SetActionOnExceed(&memory.PanicOnExceed{})
Expand Down
66 changes: 51 additions & 15 deletions expression/builtin_time.go
Original file line number Diff line number Diff line change
Expand Up @@ -1971,7 +1971,11 @@ func (b *builtinCurrentDateSig) Clone() builtinFunc {
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_curdate
func (b *builtinCurrentDateSig) evalTime(row chunk.Row) (d types.Time, isNull bool, err error) {
tz := b.ctx.GetSessionVars().Location()
year, month, day := time.Now().In(tz).Date()
var nowTs = &b.ctx.GetSessionVars().StmtCtx.NowTs
if nowTs.Equal(time.Time{}) {
*nowTs = time.Now()
}
year, month, day := nowTs.In(tz).Date()
result := types.Time{
Time: types.FromDate(year, int(month), day, 0, 0, 0, 0),
Type: mysql.TypeDate,
Expand Down Expand Up @@ -2026,7 +2030,11 @@ func (b *builtinCurrentTime0ArgSig) Clone() builtinFunc {

func (b *builtinCurrentTime0ArgSig) evalDuration(row chunk.Row) (types.Duration, bool, error) {
tz := b.ctx.GetSessionVars().Location()
dur := time.Now().In(tz).Format(types.TimeFormat)
var nowTs = &b.ctx.GetSessionVars().StmtCtx.NowTs
if nowTs.Equal(time.Time{}) {
*nowTs = time.Now()
}
dur := nowTs.In(tz).Format(types.TimeFormat)
res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, types.MinFsp)
if err != nil {
return types.Duration{}, true, errors.Trace(err)
Expand All @@ -2050,7 +2058,11 @@ func (b *builtinCurrentTime1ArgSig) evalDuration(row chunk.Row) (types.Duration,
return types.Duration{}, true, errors.Trace(err)
}
tz := b.ctx.GetSessionVars().Location()
dur := time.Now().In(tz).Format(types.TimeFSPFormat)
var nowTs = &b.ctx.GetSessionVars().StmtCtx.NowTs
if nowTs.Equal(time.Time{}) {
*nowTs = time.Now()
}
dur := nowTs.In(tz).Format(types.TimeFSPFormat)
res, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, dur, int(fsp))
if err != nil {
return types.Duration{}, true, errors.Trace(err)
Expand Down Expand Up @@ -2188,7 +2200,11 @@ func (b *builtinUTCDateSig) Clone() builtinFunc {
// evalTime evals UTC_DATE, UTC_DATE().
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-date
func (b *builtinUTCDateSig) evalTime(row chunk.Row) (types.Time, bool, error) {
year, month, day := time.Now().UTC().Date()
var nowTs = &b.ctx.GetSessionVars().StmtCtx.NowTs
if nowTs.Equal(time.Time{}) {
*nowTs = time.Now()
}
year, month, day := nowTs.UTC().Date()
result := types.Time{
Time: types.FromGoTime(time.Date(year, month, day, 0, 0, 0, 0, time.UTC)),
Type: mysql.TypeDate,
Expand Down Expand Up @@ -2244,8 +2260,12 @@ func (c *utcTimestampFunctionClass) getFunction(ctx sessionctx.Context, args []E
return sig, nil
}

func evalUTCTimestampWithFsp(fsp int) (types.Time, bool, error) {
result, err := convertTimeToMysqlTime(time.Now().UTC(), fsp)
func evalUTCTimestampWithFsp(ctx sessionctx.Context, fsp int) (types.Time, bool, error) {
var nowTs = &ctx.GetSessionVars().StmtCtx.NowTs
if nowTs.Equal(time.Time{}) {
*nowTs = time.Now()
}
result, err := convertTimeToMysqlTime(nowTs.UTC(), fsp)
if err != nil {
return types.Time{}, true, errors.Trace(err)
}
Expand Down Expand Up @@ -2277,7 +2297,7 @@ func (b *builtinUTCTimestampWithArgSig) evalTime(row chunk.Row) (types.Time, boo
return types.Time{}, true, errors.Errorf("Invalid negative %d specified, must in [0, 6].", num)
}

result, isNull, err := evalUTCTimestampWithFsp(int(num))
result, isNull, err := evalUTCTimestampWithFsp(b.ctx, int(num))
return result, isNull, errors.Trace(err)
}

Expand All @@ -2294,7 +2314,7 @@ func (b *builtinUTCTimestampWithoutArgSig) Clone() builtinFunc {
// evalTime evals UTC_TIMESTAMP().
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-timestamp
func (b *builtinUTCTimestampWithoutArgSig) evalTime(row chunk.Row) (types.Time, bool, error) {
result, isNull, err := evalUTCTimestampWithFsp(0)
result, isNull, err := evalUTCTimestampWithFsp(b.ctx, 0)
return result, isNull, errors.Trace(err)
}

Expand Down Expand Up @@ -2328,12 +2348,16 @@ func (c *nowFunctionClass) getFunction(ctx sessionctx.Context, args []Expression
}

func evalNowWithFsp(ctx sessionctx.Context, fsp int) (types.Time, bool, error) {
sysTs, err := getSystemTimestamp(ctx)
if err != nil {
return types.Time{}, true, errors.Trace(err)
var sysTs = &ctx.GetSessionVars().StmtCtx.SysTs
if sysTs.Equal(time.Time{}) {
var err error
*sysTs, err = getSystemTimestamp(ctx)
if err != nil {
return types.Time{}, true, errors.Trace(err)
}
}

result, err := convertTimeToMysqlTime(sysTs, fsp)
result, err := convertTimeToMysqlTime(*sysTs, fsp)
if err != nil {
return types.Time{}, true, errors.Trace(err)
}
Expand Down Expand Up @@ -3557,7 +3581,11 @@ func (b *builtinUnixTimestampCurrentSig) Clone() builtinFunc {
// evalInt evals a UNIX_TIMESTAMP().
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_unix-timestamp
func (b *builtinUnixTimestampCurrentSig) evalInt(row chunk.Row) (int64, bool, error) {
dec, err := goTimeToMysqlUnixTimestamp(time.Now(), 1)
var nowTs = &b.ctx.GetSessionVars().StmtCtx.NowTs
if nowTs.Equal(time.Time{}) {
*nowTs = time.Now()
}
dec, err := goTimeToMysqlUnixTimestamp(*nowTs, 1)
if err != nil {
return 0, true, errors.Trace(err)
}
Expand Down Expand Up @@ -5497,7 +5525,11 @@ func (b *builtinUTCTimeWithoutArgSig) Clone() builtinFunc {
// evalDuration evals a builtinUTCTimeWithoutArgSig.
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-time
func (b *builtinUTCTimeWithoutArgSig) evalDuration(row chunk.Row) (types.Duration, bool, error) {
v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, time.Now().UTC().Format(types.TimeFormat), 0)
var nowTs = &b.ctx.GetSessionVars().StmtCtx.NowTs
if nowTs.Equal(time.Time{}) {
*nowTs = time.Now()
}
v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFormat), 0)
return v, false, err
}

Expand All @@ -5524,7 +5556,11 @@ func (b *builtinUTCTimeWithArgSig) evalDuration(row chunk.Row) (types.Duration,
if fsp < int64(types.MinFsp) {
return types.Duration{}, true, errors.Errorf("Invalid negative %d specified, must in [0, 6].", fsp)
}
v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, time.Now().UTC().Format(types.TimeFSPFormat), int(fsp))
var nowTs = &b.ctx.GetSessionVars().StmtCtx.NowTs
if nowTs.Equal(time.Time{}) {
*nowTs = time.Now()
}
v, err := types.ParseDuration(b.ctx.GetSessionVars().StmtCtx, nowTs.UTC().Format(types.TimeFSPFormat), int(fsp))
return v, false, err
}

Expand Down
24 changes: 24 additions & 0 deletions expression/builtin_time_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -762,6 +762,11 @@ func (s *testEvaluatorSuite) TestTime(c *C) {
c.Assert(err, IsNil)
}

func resetStmtContext(ctx sessionctx.Context) {
ctx.GetSessionVars().StmtCtx.NowTs = time.Time{}
ctx.GetSessionVars().StmtCtx.SysTs = time.Time{}
}

func (s *testEvaluatorSuite) TestNowAndUTCTimestamp(c *C) {
defer testleak.AfterTest(c)()

Expand All @@ -778,6 +783,7 @@ func (s *testEvaluatorSuite) TestNowAndUTCTimestamp(c *C) {
{funcs[ast.Now], func() time.Time { return time.Now() }},
{funcs[ast.UTCTimestamp], func() time.Time { return time.Now().UTC() }},
} {
resetStmtContext(s.ctx)
f, err := x.fc.getFunction(s.ctx, s.datumsToConstants(nil))
c.Assert(err, IsNil)
v, err := evalBuiltinFunc(f, chunk.Row{})
Expand All @@ -789,6 +795,7 @@ func (s *testEvaluatorSuite) TestNowAndUTCTimestamp(c *C) {
c.Assert(strings.Contains(t.String(), "."), IsFalse)
c.Assert(ts.Sub(gotime(t, ts.Location())), LessEqual, time.Second)

resetStmtContext(s.ctx)
f, err = x.fc.getFunction(s.ctx, s.datumsToConstants(types.MakeDatums(6)))
c.Assert(err, IsNil)
v, err = evalBuiltinFunc(f, chunk.Row{})
Expand All @@ -798,11 +805,13 @@ func (s *testEvaluatorSuite) TestNowAndUTCTimestamp(c *C) {
c.Assert(strings.Contains(t.String(), "."), IsTrue)
c.Assert(ts.Sub(gotime(t, ts.Location())), LessEqual, time.Millisecond)

resetStmtContext(s.ctx)
f, err = x.fc.getFunction(s.ctx, s.datumsToConstants(types.MakeDatums(8)))
c.Assert(err, IsNil)
_, err = evalBuiltinFunc(f, chunk.Row{})
c.Assert(err, NotNil)

resetStmtContext(s.ctx)
f, err = x.fc.getFunction(s.ctx, s.datumsToConstants(types.MakeDatums(-2)))
c.Assert(err, IsNil)
_, err = evalBuiltinFunc(f, chunk.Row{})
Expand All @@ -813,6 +822,7 @@ func (s *testEvaluatorSuite) TestNowAndUTCTimestamp(c *C) {
variable.SetSessionSystemVar(s.ctx.GetSessionVars(), "time_zone", types.NewDatum("+00:00"))
variable.SetSessionSystemVar(s.ctx.GetSessionVars(), "timestamp", types.NewDatum(1234))
fc := funcs[ast.Now]
resetStmtContext(s.ctx)
f, err := fc.getFunction(s.ctx, s.datumsToConstants(nil))
c.Assert(err, IsNil)
v, err := evalBuiltinFunc(f, chunk.Row{})
Expand Down Expand Up @@ -877,6 +887,7 @@ func (s *testEvaluatorSuite) TestAddTimeSig(c *C) {

// This is a test for issue 7334
du := newDateArighmeticalUtil()
resetStmtContext(s.ctx)
now, _, err := evalNowWithFsp(s.ctx, 0)
c.Assert(err, IsNil)
res, _, err := du.add(s.ctx, now, "1", "MICROSECOND")
Expand Down Expand Up @@ -1203,6 +1214,7 @@ func (s *testEvaluatorSuite) TestUTCTime(c *C) {
}{{0, 8}, {3, 12}, {6, 15}, {-1, 0}, {7, 0}}

for _, test := range tests {
resetStmtContext(s.ctx)
f, err := fc.getFunction(s.ctx, s.datumsToConstants(types.MakeDatums(test.param)))
c.Assert(err, IsNil)
v, err := evalBuiltinFunc(f, chunk.Row{})
Expand All @@ -1229,6 +1241,7 @@ func (s *testEvaluatorSuite) TestUTCDate(c *C) {
defer testleak.AfterTest(c)()
last := time.Now().UTC()
fc := funcs[ast.UTCDate]
resetStmtContext(mock.NewContext())
f, err := fc.getFunction(mock.NewContext(), s.datumsToConstants(nil))
c.Assert(err, IsNil)
v, err := evalBuiltinFunc(f, chunk.Row{})
Expand Down Expand Up @@ -1500,6 +1513,7 @@ func (s *testEvaluatorSuite) TestTimestampDiff(c *C) {
types.NewStringDatum(test.t1),
types.NewStringDatum(test.t2),
}
resetStmtContext(s.ctx)
f, err := fc.getFunction(s.ctx, s.datumsToConstants(args))
c.Assert(err, IsNil)
d, err := evalBuiltinFunc(f, chunk.Row{})
Expand All @@ -1509,6 +1523,7 @@ func (s *testEvaluatorSuite) TestTimestampDiff(c *C) {
sc := s.ctx.GetSessionVars().StmtCtx
sc.IgnoreTruncate = true
sc.IgnoreZeroInDate = true
resetStmtContext(s.ctx)
f, err := fc.getFunction(s.ctx, s.datumsToConstants([]types.Datum{types.NewStringDatum("DAY"),
types.NewStringDatum("2017-01-00"),
types.NewStringDatum("2017-01-01")}))
Expand All @@ -1517,6 +1532,7 @@ func (s *testEvaluatorSuite) TestTimestampDiff(c *C) {
c.Assert(err, IsNil)
c.Assert(d.Kind(), Equals, types.KindNull)

resetStmtContext(s.ctx)
f, err = fc.getFunction(s.ctx, s.datumsToConstants([]types.Datum{types.NewStringDatum("DAY"),
{}, types.NewStringDatum("2017-01-01")}))
c.Assert(err, IsNil)
Expand All @@ -1528,6 +1544,7 @@ func (s *testEvaluatorSuite) TestTimestampDiff(c *C) {
func (s *testEvaluatorSuite) TestUnixTimestamp(c *C) {
// Test UNIX_TIMESTAMP().
fc := funcs[ast.UnixTimestamp]
resetStmtContext(s.ctx)
f, err := fc.getFunction(s.ctx, nil)
c.Assert(err, IsNil)
d, err := evalBuiltinFunc(f, chunk.Row{})
Expand All @@ -1537,12 +1554,14 @@ func (s *testEvaluatorSuite) TestUnixTimestamp(c *C) {

// https://github.com/pingcap/tidb/issues/2496
// Test UNIX_TIMESTAMP(NOW()).
resetStmtContext(s.ctx)
now, isNull, err := evalNowWithFsp(s.ctx, 0)
c.Assert(err, IsNil)
c.Assert(isNull, IsFalse)
n := types.Datum{}
n.SetMysqlTime(now)
args := []types.Datum{n}
resetStmtContext(s.ctx)
f, err = fc.getFunction(s.ctx, s.datumsToConstants(args))
c.Assert(err, IsNil)
d, err = evalBuiltinFunc(f, chunk.Row{})
Expand All @@ -1554,6 +1573,7 @@ func (s *testEvaluatorSuite) TestUnixTimestamp(c *C) {
// https://github.com/pingcap/tidb/issues/2852
// Test UNIX_TIMESTAMP(NULL).
args = []types.Datum{types.NewDatum(nil)}
resetStmtContext(s.ctx)
f, err = fc.getFunction(s.ctx, s.datumsToConstants(args))
c.Assert(err, IsNil)
d, err = evalBuiltinFunc(f, chunk.Row{})
Expand Down Expand Up @@ -1598,6 +1618,7 @@ func (s *testEvaluatorSuite) TestUnixTimestamp(c *C) {
fmt.Printf("Begin Test %v\n", test)
expr := s.datumsToConstants([]types.Datum{test.input})
expr[0].GetType().Decimal = test.inputDecimal
resetStmtContext(s.ctx)
f, err := fc.getFunction(s.ctx, expr)
c.Assert(err, IsNil, Commentf("%+v", test))
d, err := evalBuiltinFunc(f, chunk.Row{})
Expand Down Expand Up @@ -1681,6 +1702,7 @@ func (s *testEvaluatorSuite) TestTimestamp(c *C) {
}
fc := funcs[ast.Timestamp]
for _, test := range tests {
resetStmtContext(s.ctx)
f, err := fc.getFunction(s.ctx, s.datumsToConstants(test.t))
c.Assert(err, IsNil)
d, err := evalBuiltinFunc(f, chunk.Row{})
Expand All @@ -1690,6 +1712,7 @@ func (s *testEvaluatorSuite) TestTimestamp(c *C) {
}

nilDatum := types.NewDatum(nil)
resetStmtContext(s.ctx)
f, err := fc.getFunction(s.ctx, s.datumsToConstants([]types.Datum{nilDatum}))
c.Assert(err, IsNil)
d, err := evalBuiltinFunc(f, chunk.Row{})
Expand Down Expand Up @@ -2357,6 +2380,7 @@ func (s *testEvaluatorSuite) TestWithTimeZone(c *C) {

for _, t := range tests {
now := time.Now().In(sv.TimeZone)
resetStmtContext(s.ctx)
f, err := funcs[t.method].getFunction(s.ctx, s.datumsToConstants(t.Input))
c.Assert(err, IsNil)
d, err := evalBuiltinFunc(f, chunk.Row{})
Expand Down
40 changes: 23 additions & 17 deletions expression/function_traits.go
Original file line number Diff line number Diff line change
Expand Up @@ -19,23 +19,12 @@ import (

// UnCacheableFunctions stores functions which can not be cached to plan cache.
var UnCacheableFunctions = map[string]struct{}{
ast.Now: {},
ast.CurrentTimestamp: {},
ast.UTCTime: {},
ast.Curtime: {},
ast.CurrentTime: {},
ast.UTCTimestamp: {},
ast.UnixTimestamp: {},
ast.Sysdate: {},
ast.Curdate: {},
ast.CurrentDate: {},
ast.UTCDate: {},
ast.Database: {},
ast.CurrentUser: {},
ast.User: {},
ast.ConnectionID: {},
ast.LastInsertId: {},
ast.Version: {},
ast.Database: {},
ast.CurrentUser: {},
ast.User: {},
ast.ConnectionID: {},
ast.LastInsertId: {},
ast.Version: {},
}

// unFoldableFunctions stores functions which can not be folded duration constant folding stage.
Expand All @@ -52,6 +41,23 @@ var unFoldableFunctions = map[string]struct{}{
ast.GetParam: {},
}

// DeferredFunctions stores non-deterministic functions, which can be deferred only when the plan cache is enabled.
var DeferredFunctions = map[string]struct{}{
ast.Now: {},
ast.CurrentTimestamp: {},
ast.UTCTime: {},
ast.Curtime: {},
ast.CurrentTime: {},
ast.UTCTimestamp: {},
ast.UnixTimestamp: {},
ast.Sysdate: {},
ast.Curdate: {},
ast.CurrentDate: {},
ast.UTCDate: {},
ast.Rand: {},
ast.UUID: {},
}

// inequalFunctions stores functions which cannot be propagated from column equal condition.
var inequalFunctions = map[string]struct{}{
ast.IsNull: {},
Expand Down
Loading

0 comments on commit 5374ff8

Please sign in to comment.