Skip to content

Commit

Permalink
*: add builtin function UTC_TIMESTAMP (#2592)
Browse files Browse the repository at this point in the history
* *: add builtin function UTC_TIMESTAMP
  • Loading branch information
zyguan authored and ngaut committed Feb 5, 2017
1 parent c131b3c commit 395a9c6
Show file tree
Hide file tree
Showing 9 changed files with 117 additions and 43 deletions.
1 change: 1 addition & 0 deletions ast/functions.go
Original file line number Diff line number Diff line change
Expand Up @@ -122,6 +122,7 @@ const (
Timestamp = "timestamp"
TimestampDiff = "timestampdiff"
UTCDate = "utc_date"
UTCTimestamp = "utc_timestamp"
UnixTimestamp = "unix_timestamp"
Week = "week"
Weekday = "weekday"
Expand Down
1 change: 1 addition & 0 deletions expression/builtin.go
Original file line number Diff line number Diff line change
Expand Up @@ -182,6 +182,7 @@ var funcs = map[string]functionClass{
ast.Time: &timeFunctionClass{baseFunctionClass{ast.Time, 1, 1}},
ast.Timestamp: &timestampFunctionClass{baseFunctionClass{ast.Timestamp, 1, 2}},
ast.UTCDate: &utcDateFunctionClass{baseFunctionClass{ast.UTCDate, 0, 0}},
ast.UTCTimestamp: &utcTimestampFunctionClass{baseFunctionClass{ast.UnixTimestamp, 0, 1}},
ast.Week: &weekFunctionClass{baseFunctionClass{ast.Week, 1, 2}},
ast.Weekday: &weekDayFunctionClass{baseFunctionClass{ast.Weekday, 1, 1}},
ast.WeekOfYear: &weekOfYearFunctionClass{baseFunctionClass{ast.WeekOfYear, 1, 1}},
Expand Down
70 changes: 54 additions & 16 deletions expression/builtin_time.go
Original file line number Diff line number Diff line change
Expand Up @@ -60,6 +60,7 @@ var (
_ functionClass = &currentTimeFunctionClass{}
_ functionClass = &timeFunctionClass{}
_ functionClass = &utcDateFunctionClass{}
_ functionClass = &utcTimestampFunctionClass{}
_ functionClass = &extractFunctionClass{}
_ functionClass = &arithmeticFunctionClass{}
_ functionClass = &unixTimestampFunctionClass{}
Expand Down Expand Up @@ -94,12 +95,26 @@ var (
_ builtinFunc = &builtinCurrentTimeSig{}
_ builtinFunc = &builtinTimeSig{}
_ builtinFunc = &builtinUTCDateSig{}
_ builtinFunc = &builtinUTCTimestampSig{}
_ builtinFunc = &builtinExtractSig{}
_ builtinFunc = &builtinArithmeticSig{}
_ builtinFunc = &builtinUnixTimestampSig{}
_ builtinFunc = &builtinTimestampSig{}
)

func convertTimeToMysqlTime(t time.Time, fsp int) (types.Time, error) {
tr, err := types.RoundFrac(t, int(fsp))
if err != nil {
return types.Time{}, errors.Trace(err)
}

return types.Time{
Time: types.FromGoTime(tr),
Type: mysql.TypeDatetime,
Fsp: fsp,
}, nil
}

func convertToTime(sc *variable.StatementContext, arg types.Datum, tp byte) (d types.Datum, err error) {
f := types.NewFieldType(tp)
f.Decimal = types.MaxFsp
Expand Down Expand Up @@ -528,24 +543,15 @@ func builtinNow(args []types.Datum, ctx context.Context) (d types.Datum, err err
sc := ctx.GetSessionVars().StmtCtx
if len(args) == 1 && !args[0].IsNull() {
if fsp, err = checkFsp(sc, args[0]); err != nil {
d.SetNull()
return d, errors.Trace(err)
}
}

tr, err := types.RoundFrac(time.Now(), int(fsp))
t, err := convertTimeToMysqlTime(time.Now(), fsp)
if err != nil {
d.SetNull()
return d, errors.Trace(err)
}

t := types.Time{
Time: types.FromGoTime(tr),
Type: mysql.TypeDatetime,
// set unspecified for later round
Fsp: fsp,
}

d.SetMysqlTime(t)
return d, nil
}
Expand Down Expand Up @@ -952,16 +958,12 @@ func (b *builtinFromUnixTimeSig) eval(row []types.Datum) (d types.Datum, err err
if fracDigitsNumber > types.MaxFsp {
fsp = types.MaxFsp
}
tr, err := types.RoundFrac(time.Unix(integralPart, fractionalPart), fsp)

t, err := convertTimeToMysqlTime(time.Unix(integralPart, fractionalPart), fsp)
if err != nil {
return d, errors.Trace(err)
}

t := types.Time{
Time: types.FromGoTime(tr),
Type: mysql.TypeDatetime,
Fsp: fsp,
}
if args[0].Kind() == types.KindString { // Keep consistent with MySQL.
t.Fsp = types.MaxFsp
}
Expand Down Expand Up @@ -1145,6 +1147,42 @@ func (b *builtinUTCDateSig) eval(_ []types.Datum) (d types.Datum, err error) {
return d, nil
}

type utcTimestampFunctionClass struct {
baseFunctionClass
}

func (c *utcTimestampFunctionClass) getFunction(args []Expression, ctx context.Context) (builtinFunc, error) {
return &builtinUTCTimestampSig{newBaseBuiltinFunc(args, ctx)}, errors.Trace(c.verifyArgs(args))
}

type builtinUTCTimestampSig struct {
baseBuiltinFunc
}

// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_utc-timestamp
func (b *builtinUTCTimestampSig) eval(row []types.Datum) (d types.Datum, err error) {
args, err := b.evalArgs(row)
if err != nil {
return types.Datum{}, errors.Trace(err)
}

fsp := 0
sc := b.ctx.GetSessionVars().StmtCtx
if len(args) == 1 && !args[0].IsNull() {
if fsp, err = checkFsp(sc, args[0]); err != nil {
return d, errors.Trace(err)
}
}

t, err := convertTimeToMysqlTime(time.Now().UTC(), fsp)
if err != nil {
return d, errors.Trace(err)
}

d.SetMysqlTime(t)
return d, nil
}

type extractFunctionClass struct {
baseFunctionClass
}
Expand Down
66 changes: 42 additions & 24 deletions expression/builtin_time_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -418,34 +418,52 @@ func (s *testEvaluatorSuite) TestClock(c *C) {
}
}

func (s *testEvaluatorSuite) TestNow(c *C) {
func (s *testEvaluatorSuite) TestNowAndUTCTimestamp(c *C) {
defer testleak.AfterTest(c)()
fc := funcs[ast.Now]
f, err := fc.getFunction(datumsToConstants(nil), s.ctx)
c.Assert(err, IsNil)
v, err := f.eval(nil)
c.Assert(err, IsNil)
t := v.GetMysqlTime()
// we canot use a constant value to check now, so here
// just to check whether has fractional seconds part.
c.Assert(strings.Contains(t.String(), "."), IsFalse)

f, err = fc.getFunction(datumsToConstants(types.MakeDatums(6)), s.ctx)
c.Assert(err, IsNil)
v, err = f.eval(nil)
c.Assert(err, IsNil)
t = v.GetMysqlTime()
c.Assert(strings.Contains(t.String(), "."), IsTrue)
gotime := func(t types.Time, l *time.Location) time.Time {
tt, err := t.Time.GoTime(l)
c.Assert(err, IsNil)
return tt
}

f, err = fc.getFunction(datumsToConstants(types.MakeDatums(8)), s.ctx)
c.Assert(err, IsNil)
_, err = f.eval(nil)
c.Assert(err, NotNil)
for _, x := range []struct {
fc functionClass
now func() time.Time
}{
{funcs[ast.Now], func() time.Time { return time.Now() }},
{funcs[ast.UTCTimestamp], func() time.Time { return time.Now().UTC() }},
} {
f, err := x.fc.getFunction(datumsToConstants(nil), s.ctx)
c.Assert(err, IsNil)
v, err := f.eval(nil)
ts := x.now()
c.Assert(err, IsNil)
t := v.GetMysqlTime()
// we canot use a constant value to check timestamp funcs, so here
// just to check the fractional seconds part and the time delta.
c.Assert(strings.Contains(t.String(), "."), IsFalse)
c.Assert(ts.Sub(gotime(t, ts.Location())), LessEqual, time.Second)

f, err = fc.getFunction(datumsToConstants(types.MakeDatums(-2)), s.ctx)
c.Assert(err, IsNil)
_, err = f.eval(nil)
c.Assert(err, NotNil)
f, err = x.fc.getFunction(datumsToConstants(types.MakeDatums(6)), s.ctx)
c.Assert(err, IsNil)
v, err = f.eval(nil)
ts = x.now()
c.Assert(err, IsNil)
t = v.GetMysqlTime()
c.Assert(strings.Contains(t.String(), "."), IsTrue)
c.Assert(ts.Sub(gotime(t, ts.Location())), LessEqual, time.Millisecond)

f, err = x.fc.getFunction(datumsToConstants(types.MakeDatums(8)), s.ctx)
c.Assert(err, IsNil)
_, err = f.eval(nil)
c.Assert(err, NotNil)

f, err = x.fc.getFunction(datumsToConstants(types.MakeDatums(-2)), s.ctx)
c.Assert(err, IsNil)
_, err = f.eval(nil)
c.Assert(err, NotNil)
}
}

func (s *testEvaluatorSuite) TestSysDate(c *C) {
Expand Down
1 change: 1 addition & 0 deletions parser/misc.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,6 +190,7 @@ var tokenMap = map[string]int{
"CROSS": cross,
"CURDATE": curDate,
"UTC_DATE": utcDate,
"UTC_TIMESTAMP": utcTimestamp,
"CURRENT_DATE": currentDate,
"CURTIME": curTime,
"CURRENT_TIME": currentTime,
Expand Down
11 changes: 10 additions & 1 deletion parser/parser.y
Original file line number Diff line number Diff line change
Expand Up @@ -201,6 +201,7 @@ import (
use "USE"
using "USING"
utcDate "UTC_DATE"
utcTimestamp "UTC_TIMESTAMP"
values "VALUES"
varcharType "VARCHAR"
varbinaryType "VARBINARY"
Expand Down Expand Up @@ -2149,7 +2150,7 @@ ReservedKeyword:
| "SCHEMA" | "SCHEMAS" | "SECOND_MICROSECOND" | "SELECT" | "SET" | "SHOW" | "SMALLINT"
| "STARTING" | "TABLE" | "TERMINATED" | "THEN" | "TINYBLOB" | "TINYINT" | "TINYTEXT" | "TO"
| "TRAILING" | "TRUE" | "UNION" | "UNIQUE" | "UNLOCK" | "UNSIGNED"
| "UPDATE" | "USE" | "USING" | "UTC_DATE" | "VALUES" | "VARBINARY" | "VARCHAR"
| "UPDATE" | "USE" | "USING" | "UTC_DATE" | "UTC_TIMESTAMP" | "VALUES" | "VARBINARY" | "VARCHAR"
| "WHEN" | "WHERE" | "WRITE" | "XOR" | "YEAR_MONTH" | "ZEROFILL"
/*
| "DELAYED" | "HIGH_PRIORITY" | "LOW_PRIORITY"| "WITH"
Expand Down Expand Up @@ -3082,6 +3083,14 @@ FunctionCallNonKeyword:
{
$$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
}
| "UTC_TIMESTAMP" FuncDatetimePrec
{
args := []ast.ExprNode{}
if $2 != nil {
args = append(args, $2.(ast.ExprNode))
}
$$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: args}
}
| "WEEKDAY" '(' Expression ')'
{
$$ = &ast.FuncCallExpr{FnName: model.NewCIStr($1), Args: []ast.ExprNode{$3.(ast.ExprNode)}}
Expand Down
5 changes: 5 additions & 0 deletions parser/parser_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -584,6 +584,11 @@ func (s *testParserSuite) TestBuiltin(c *C) {
{"select curtime()", true},
{"select curtime(6)", true},

// select utc_timestamp
{"select utc_timestamp", true},
{"select utc_timestamp()", true},
{"select utc_timestamp(6)", true},

// for microsecond, second, minute, hour
{"SELECT MICROSECOND('2009-12-31 23:59:59.000010');", true},
{"SELECT SECOND('10:05:03');", true},
Expand Down
4 changes: 2 additions & 2 deletions plan/typeinferer.go
Original file line number Diff line number Diff line change
Expand Up @@ -302,13 +302,13 @@ func (v *typeInferrer) handleFuncCallExpr(x *ast.FuncCallExpr) {
case "curtime", "current_time", "timediff":
tp = types.NewFieldType(mysql.TypeDuration)
tp.Decimal = v.getFsp(x)
case "current_timestamp", "date_add", "date_sub", "adddate", "subdate", "timestamp":
case "date_add", "date_sub", "adddate", "subdate", "timestamp":
tp = types.NewFieldType(mysql.TypeDatetime)
case "microsecond", "second", "minute", "hour", "day", "week", "month", "year",
"dayofweek", "dayofmonth", "dayofyear", "weekday", "weekofyear", "yearweek", "datediff",
"found_rows", "length", "extract", "locate", "unix_timestamp":
tp = types.NewFieldType(mysql.TypeLonglong)
case "now", "sysdate":
case "now", "sysdate", "current_timestamp", "utc_timestamp":
tp = types.NewFieldType(mysql.TypeDatetime)
tp.Decimal = v.getFsp(x)
case "from_unixtime":
Expand Down
1 change: 1 addition & 0 deletions plan/typeinferer_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -107,6 +107,7 @@ func (ts *testTypeInferrerSuite) TestInferType(c *C) {
{"current_time()", mysql.TypeDuration, charset.CharsetBin},
{"curtime()", mysql.TypeDuration, charset.CharsetBin},
{"current_timestamp()", mysql.TypeDatetime, charset.CharsetBin},
{"utc_timestamp()", mysql.TypeDatetime, charset.CharsetBin},
{"microsecond('2009-12-31 23:59:59.000010')", mysql.TypeLonglong, charset.CharsetBin},
{"second('2009-12-31 23:59:59.000010')", mysql.TypeLonglong, charset.CharsetBin},
{"minute('2009-12-31 23:59:59.000010')", mysql.TypeLonglong, charset.CharsetBin},
Expand Down

0 comments on commit 395a9c6

Please sign in to comment.