Skip to content

Commit

Permalink
expression: rewrite builtin function: UNIX_TIMESTAMP (pingcap#4297)
Browse files Browse the repository at this point in the history
  • Loading branch information
breezewish authored and hanfei1991 committed Aug 31, 2017
1 parent 334dd05 commit 861d844
Showing 4 changed files with 239 additions and 64 deletions.
155 changes: 108 additions & 47 deletions expression/builtin_time.go
Original file line number Diff line number Diff line change
@@ -141,7 +141,9 @@ var (
_ builtinFunc = &builtinUTCTimestampWithoutArgSig{}
_ builtinFunc = &builtinExtractSig{}
_ builtinFunc = &builtinArithmeticSig{}
_ builtinFunc = &builtinUnixTimestampSig{}
_ builtinFunc = &builtinUnixTimestampCurrentSig{}
_ builtinFunc = &builtinUnixTimestampIntSig{}
_ builtinFunc = &builtinUnixTimestampDecSig{}
_ builtinFunc = &builtinAddTimeSig{}
_ builtinFunc = &builtinConvertTzSig{}
_ builtinFunc = &builtinMakeDateSig{}
@@ -2009,64 +2011,123 @@ func (c *unixTimestampFunctionClass) getFunction(args []Expression, ctx context.
if err := c.verifyArgs(args); err != nil {
return nil, errors.Trace(err)
}
sig := &builtinUnixTimestampSig{newBaseBuiltinFunc(args, ctx)}
var argTps []evalTp
var retTp evalTp
var retFLen, retDecimal int

if len(args) == 0 {
retTp, retDecimal = tpInt, 0
} else {
argTps = []evalTp{tpDatetime}
argType := args[0].GetType()
argEvaltp := fieldTp2EvalTp(argType)
if argEvaltp == tpString {
// Treat tpString as unspecified decimal.
retDecimal = types.UnspecifiedLength
} else {
retDecimal = argType.Decimal
}
if retDecimal > 6 || retDecimal == types.UnspecifiedLength {
retDecimal = 6
}
if retDecimal == 0 {
retTp = tpInt
} else {
retTp = tpDecimal
}
}
if retTp == tpInt {
retFLen = 11
} else if retTp == tpDecimal {
retFLen = 12 + retDecimal
} else {
panic("Unexpected retTp")
}

bf := newBaseBuiltinFuncWithTp(args, ctx, retTp, argTps...)
bf.deterministic = false
bf.tp.Flen = retFLen
bf.tp.Decimal = retDecimal

var sig builtinFunc
if len(args) == 0 {
sig = &builtinUnixTimestampCurrentSig{baseIntBuiltinFunc{bf}}
} else if retTp == tpInt {
sig = &builtinUnixTimestampIntSig{baseIntBuiltinFunc{bf}}
} else if retTp == tpDecimal {
sig = &builtinUnixTimestampDecSig{baseDecimalBuiltinFunc{bf}}
} else {
panic("Unexpected retTp")
}

return sig.setSelf(sig), nil
}

type builtinUnixTimestampSig struct {
baseBuiltinFunc
// goTimeToMysqlUnixTimestamp converts go time into MySQL's Unix timestamp.
// MySQL's Unix timestamp ranges in int32. Values out of range should be rewritten to 0.
func goTimeToMysqlUnixTimestamp(t time.Time, decimal int) *types.MyDecimal {
nanoSeconds := t.UnixNano()
if nanoSeconds < 0 || (nanoSeconds/1e3) >= (math.MaxInt32+1)*1e6 {
return new(types.MyDecimal)
}
dec := new(types.MyDecimal)
// Here we don't use float to prevent precision lose.
dec.FromInt(nanoSeconds)
dec.Shift(-9)
dec.Round(dec, decimal, types.ModeHalfEven)
return dec
}

type builtinUnixTimestampCurrentSig struct {
baseIntBuiltinFunc
}

// eval evals a builtinUnixTimestampSig.
// 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 *builtinUnixTimestampSig) eval(row []types.Datum) (d types.Datum, err error) {
args, err := b.evalArgs(row)
if err != nil {
return d, errors.Trace(err)
}
if len(args) == 0 {
now := time.Now().Unix()
d.SetInt64(now)
return
}
func (b *builtinUnixTimestampCurrentSig) evalInt(row []types.Datum) (int64, bool, error) {
dec := goTimeToMysqlUnixTimestamp(time.Now(), 1)
intVal, _ := dec.ToInt() // Ignore truncate errors.
return intVal, false, nil
}

var (
t types.Time
t1 time.Time
)
switch args[0].Kind() {
case types.KindString:
t, err = types.ParseTime(args[0].GetString(), mysql.TypeDatetime, types.MaxFsp)
if err != nil {
return d, errors.Trace(err)
}
case types.KindInt64, types.KindUint64:
t, err = types.ParseTimeFromInt64(args[0].GetInt64())
if err != nil {
return d, errors.Trace(err)
}
case types.KindMysqlTime:
t = args[0].GetMysqlTime()
case types.KindNull:
return
default:
return d, errors.Errorf("Unknown args type for unix_timestamp %d", args[0].Kind())
}
type builtinUnixTimestampIntSig struct {
baseIntBuiltinFunc
}

t1, err = t.Time.GoTime(getTimeZone(b.ctx))
// evalInt evals a UNIX_TIMESTAMP(time).
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_unix-timestamp
func (b *builtinUnixTimestampIntSig) evalInt(row []types.Datum) (int64, bool, error) {
val, isNull, err := b.args[0].EvalTime(row, b.getCtx().GetSessionVars().StmtCtx)
if isNull || err != nil {
// Return 0 for invalid date time.
return 0, isNull, nil
}
t, err := val.Time.GoTime(getTimeZone(b.getCtx()))
if err != nil {
d.SetInt64(0)
return d, nil
return 0, false, nil
}
dec := goTimeToMysqlUnixTimestamp(t, 1)
intVal, _ := dec.ToInt() // Ignore truncate errors.
return intVal, false, nil
}

if t.Time.Microsecond() > 0 {
var dec types.MyDecimal
dec.FromFloat64(float64(t1.Unix()) + float64(t.Time.Microsecond())/1e6)
d.SetMysqlDecimal(&dec)
} else {
d.SetInt64(t1.Unix())
type builtinUnixTimestampDecSig struct {
baseDecimalBuiltinFunc
}

// evalDecimal evals a UNIX_TIMESTAMP(time).
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_unix-timestamp
func (b *builtinUnixTimestampDecSig) evalDecimal(row []types.Datum) (*types.MyDecimal, bool, error) {
val, isNull, err := b.args[0].EvalTime(row, b.getCtx().GetSessionVars().StmtCtx)
if isNull || err != nil {
// Return 0 for invalid date time.
return new(types.MyDecimal), isNull, nil
}
return
t, err := val.Time.GoTime(getTimeZone(b.getCtx()))
if err != nil {
return new(types.MyDecimal), false, nil
}
return goTimeToMysqlUnixTimestamp(t, b.tp.Decimal), false, nil
}

type timestampFunctionClass struct {
66 changes: 49 additions & 17 deletions expression/builtin_time_test.go
Original file line number Diff line number Diff line change
@@ -14,6 +14,7 @@
package expression

import (
"fmt"
"math"
"strings"
"time"
@@ -1369,16 +1370,18 @@ func (s *testEvaluatorSuite) TestTimestampDiff(c *C) {
}

func (s *testEvaluatorSuite) TestUnixTimestamp(c *C) {
// Test UNIX_TIMESTAMP().
fc := funcs[ast.UnixTimestamp]
f, err := fc.getFunction(nil, s.ctx)
c.Assert(err, IsNil)
c.Assert(f.isDeterministic(), IsFalse)
d, err := f.eval(nil)
c.Assert(err, IsNil)
c.Assert(d.GetInt64()-time.Now().Unix(), GreaterEqual, int64(-1))
c.Assert(d.GetInt64()-time.Now().Unix(), LessEqual, int64(1))

// Test case for https://github.com/pingcap/tidb/issues/2496
// select unix_timestamp(now());
// https://github.com/pingcap/tidb/issues/2496
// Test UNIX_TIMESTAMP(NOW()).
now, isNull, err := evalNowWithFsp(s.ctx, 0)
c.Assert(err, IsNil)
c.Assert(isNull, IsFalse)
@@ -1389,38 +1392,67 @@ func (s *testEvaluatorSuite) TestUnixTimestamp(c *C) {
c.Assert(err, IsNil)
d, err = f.eval(nil)
c.Assert(err, IsNil)
c.Assert(d.GetInt64()-time.Now().Unix(), GreaterEqual, int64(-1))
c.Assert(d.GetInt64()-time.Now().Unix(), LessEqual, int64(1))
val, _ := d.GetMysqlDecimal().ToInt()
c.Assert(val-time.Now().Unix(), GreaterEqual, int64(-1))
c.Assert(val-time.Now().Unix(), LessEqual, int64(1))

// Test case for https://github.com/pingcap/tidb/issues/2852
// select UNIX_TIMESTAMP(null);
// https://github.com/pingcap/tidb/issues/2852
// Test UNIX_TIMESTAMP(NULL).
args = []types.Datum{types.NewDatum(nil)}
f, err = fc.getFunction(datumsToConstants(args), s.ctx)
c.Assert(err, IsNil)
c.Assert(f.isDeterministic(), IsFalse)
d, err = f.eval(nil)
c.Assert(err, IsNil)
c.Assert(d.IsNull(), Equals, true)

// Set the time_zone variable, because UnixTimestamp() result depends on it.
s.ctx.GetSessionVars().TimeZone = time.UTC
tests := []struct {
input types.Datum
expect string
inputDecimal int
input types.Datum
expectKind byte
expect string
}{
{types.NewIntDatum(20151113102019), "1447410019"},
{types.NewStringDatum("2015-11-13 10:20:19"), "1447410019"},
{types.NewStringDatum("2015-11-13 10:20:19.012"), "1447410019.012"},
{types.NewStringDatum("2017-00-02"), "0"},
{0, types.NewIntDatum(151113), types.KindInt64, "1447372800"}, // YYMMDD
// TODO: Uncomment the line below after fixing #4232
// {5, types.NewFloat64Datum(151113.12345), types.KindMysqlDecimal, "1447372800.00000"}, // YYMMDD
{0, types.NewIntDatum(20151113), types.KindInt64, "1447372800"}, // YYYYMMDD
// TODO: Uncomment the line below after fixing #4232
// {5, types.NewFloat64Datum(20151113.12345), types.KindMysqlDecimal, "1447372800.00000"}, // YYYYMMDD
{0, types.NewIntDatum(151113102019), types.KindInt64, "1447410019"}, // YYMMDDHHMMSS
{0, types.NewFloat64Datum(151113102019), types.KindInt64, "1447410019"}, // YYMMDDHHMMSS
{2, types.NewFloat64Datum(151113102019.12), types.KindMysqlDecimal, "1447410019.12"}, // YYMMDDHHMMSS
{0, types.NewDecimalDatum(types.NewDecFromStringForTest("151113102019")), types.KindInt64, "1447410019"}, // YYMMDDHHMMSS
{2, types.NewDecimalDatum(types.NewDecFromStringForTest("151113102019.12")), types.KindMysqlDecimal, "1447410019.12"}, // YYMMDDHHMMSS
{7, types.NewDecimalDatum(types.NewDecFromStringForTest("151113102019.1234567")), types.KindMysqlDecimal, "1447410019.123457"}, // YYMMDDHHMMSS
{0, types.NewIntDatum(20151113102019), types.KindInt64, "1447410019"}, // YYYYMMDDHHMMSS
// TODO: for string literal inputs as below, fsp should be based on user input.
{0, types.NewStringDatum("2015-11-13 10:20:19"), types.KindMysqlDecimal, "1447410019.000000"},
{0, types.NewStringDatum("2015-11-13 10:20:19.012"), types.KindMysqlDecimal, "1447410019.012000"},
{0, types.NewStringDatum("1970-01-01 00:00:00"), types.KindMysqlDecimal, "0.000000"}, // Min timestamp
{0, types.NewStringDatum("2038-01-19 03:14:07.999999"), types.KindMysqlDecimal, "2147483647.999999"}, // Max timestamp
{0, types.NewStringDatum("2017-00-02"), types.KindMysqlDecimal, "0"}, // Invalid date
{0, types.NewStringDatum("1969-12-31 23:59:59.999999"), types.KindMysqlDecimal, "0"}, // Invalid timestamp
{0, types.NewStringDatum("2038-01-19 03:14:08"), types.KindMysqlDecimal, "0"}, // Invalid timestamp
// Below tests irregular inputs.
{0, types.NewIntDatum(0), types.KindInt64, "0"},
{0, types.NewIntDatum(-1), types.KindInt64, "0"},
{0, types.NewIntDatum(12345), types.KindInt64, "0"},
}

for _, test := range tests {
f, err := fc.getFunction(datumsToConstants([]types.Datum{test.input}), s.ctx)
c.Assert(err, IsNil)
fmt.Printf("Begin Test %v\n", test)
expr := datumsToConstants([]types.Datum{test.input})
expr[0].GetType().Decimal = test.inputDecimal
f, err := fc.getFunction(expr, s.ctx)
c.Assert(err, IsNil, Commentf("%+v", test))
d, err := f.eval(nil)
c.Assert(err, IsNil)
c.Assert(err, IsNil, Commentf("%+v", test))
c.Assert(d.Kind(), Equals, test.expectKind, Commentf("%+v", test))
str, err := d.ToString()
c.Assert(err, IsNil)
c.Assert(str, Equals, test.expect)
c.Assert(err, IsNil, Commentf("%+v", test))
c.Assert(str, Equals, test.expect, Commentf("%+v", test))
}
}

59 changes: 59 additions & 0 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
@@ -1299,6 +1299,65 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) {
_, err = tk.Exec(`delete from t where a = dayOfYear(123)`)
c.Assert(terror.ErrorEqual(err, types.ErrInvalidTimeFormat), IsTrue)

tk.MustExec("set sql_mode = ''")

// for unix_timestamp
tk.MustExec("SET time_zone = '+00:00';")
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(151113);")
result.Check(testkit.Rows("1447372800"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(20151113);")
result.Check(testkit.Rows("1447372800"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(151113102019);")
result.Check(testkit.Rows("1447410019"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(151113102019e0);")
result.Check(testkit.Rows("1447410019.000000"))
// See https://github.com/pingcap/tidb/issues/4296.
// result = tk.MustQuery("SELECT UNIX_TIMESTAMP(15111310201912e-2);")
// result.Check(testkit.Rows("1447410019.119995"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(151113102019.12);")
result.Check(testkit.Rows("1447410019.12"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(151113102019.1234567);")
result.Check(testkit.Rows("1447410019.123457"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(20151113102019);")
result.Check(testkit.Rows("1447410019"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2015-11-13 10:20:19');")
result.Check(testkit.Rows("1447410019.000000"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2015-11-13 10:20:19.012');")
result.Check(testkit.Rows("1447410019.012000"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-01-01 00:00:00');")
result.Check(testkit.Rows("0.000000"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1969-12-31 23:59:59');")
result.Check(testkit.Rows("0"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-13-01 00:00:00');")
result.Check(testkit.Rows("0"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2038-01-19 03:14:07.999999');")
result.Check(testkit.Rows("2147483647.999999"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2038-01-19 03:14:08');")
result.Check(testkit.Rows("0"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(0);")
result.Check(testkit.Rows("0"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(-1);")
result.Check(testkit.Rows("0"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(12345);")
result.Check(testkit.Rows("0"))
// Test different time zone.
tk.MustExec("SET time_zone = '+08:00';")
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-01-01 00:00:00');")
result.Check(testkit.Rows("0"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('1970-01-01 08:00:00');")
result.Check(testkit.Rows("0.000000"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2015-11-13 18:20:19.012');")
result.Check(testkit.Rows("1447410019.012000"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP('2038-01-19 11:14:07.999999');")
result.Check(testkit.Rows("2147483647.999999"))

result = tk.MustQuery("SELECT TIME_FORMAT('bad string', '%H:%i:%s %p');")
result.Check(testkit.Rows("00:00:00 AM"))
result = tk.MustQuery("SELECT TIME_FORMAT(null, '%H:%i:%s %p');")
result.Check(testkit.Rows("<nil>"))
result = tk.MustQuery("SELECT TIME_FORMAT(123, '%H:%i:%s %p');")
result.Check(testkit.Rows("00:01:23 AM"))

// for monthname
tk.MustExec(`drop table if exists t`)
tk.MustExec(`create table t(a varchar(10))`)
23 changes: 23 additions & 0 deletions plan/typeinfer_test.go
Original file line number Diff line number Diff line change
@@ -1092,6 +1092,29 @@ func (s *testPlanSuite) createTestCase4TimeFuncs() []typeInferTestCase {
{"to_seconds(c_char)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0},
{"to_days(c_char)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0},

// TODO: Some of the tests are commented out because of #4233.
// {"unix_timestamp(c_int_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0},
// {"unix_timestamp(c_bigint_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0},
{"unix_timestamp(c_float_d )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
{"unix_timestamp(c_double_d )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
{"unix_timestamp(c_decimal )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"unix_timestamp(c_decimal_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0},
{"unix_timestamp(c_datetime )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 14, 2},
{"unix_timestamp(c_datetime_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0},
{"unix_timestamp(c_time )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 15, 3},
{"unix_timestamp(c_time_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0},
{"unix_timestamp(c_timestamp )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 16, 4},
// {"unix_timestamp(c_timestamp_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 11, 0},
{"unix_timestamp(c_char )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
{"unix_timestamp(c_varchar )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
{"unix_timestamp(c_text_d )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
{"unix_timestamp(c_binary )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
{"unix_timestamp(c_varbinary )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
{"unix_timestamp(c_blob_d )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
{"unix_timestamp(c_set )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
{"unix_timestamp(c_enum )", mysql.TypeNewDecimal, charset.CharsetBin, mysql.BinaryFlag, 18, 6},
// TODO: Add string literal tests for UNIX_TIMESTAMP. UNIX_TIMESTAMP respects the fsp in string literals.

{"timestampdiff(MONTH, c_datetime, c_datetime)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0},
{"timestampdiff(QuarteR, c_char, c_varchar)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0},
{"timestampdiff(second, c_int_d, c_bchar)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 20, 0},

0 comments on commit 861d844

Please sign in to comment.