Skip to content

Commit

Permalink
expression: rewrite builtin function: TIME_TO_SEC, SEC_TO_TIME (pingc…
Browse files Browse the repository at this point in the history
  • Loading branch information
breezewish authored Sep 5, 2017
1 parent 778b221 commit 0513ab3
Show file tree
Hide file tree
Showing 5 changed files with 180 additions and 99 deletions.
99 changes: 45 additions & 54 deletions expression/builtin_time.go
Original file line number Diff line number Diff line change
Expand Up @@ -2803,25 +2803,41 @@ func (c *secToTimeFunctionClass) getFunction(ctx context.Context, args []Express
if err := c.verifyArgs(args); err != nil {
return nil, errors.Trace(err)
}
sig := &builtinSecToTimeSig{newBaseBuiltinFunc(args, ctx)}
var retFlen, retFsp int
argType := args[0].GetType()
argEvalTp := fieldTp2EvalTp(argType)
if argEvalTp == tpString {
retFsp = types.UnspecifiedLength
} else {
retFsp = argType.Decimal
}
if retFsp > types.MaxFsp || retFsp == types.UnspecifiedLength {
retFsp = types.MaxFsp
} else if retFsp < types.MinFsp {
retFsp = types.MinFsp
}
retFlen = 10
if retFsp > 0 {
retFlen += 1 + retFsp
}
bf := newBaseBuiltinFuncWithTp(args, ctx, tpDuration, tpReal)
bf.tp.Flen, bf.tp.Decimal = retFlen, retFsp
sig := &builtinSecToTimeSig{baseDurationBuiltinFunc{bf}}
return sig.setSelf(sig), nil
}

type builtinSecToTimeSig struct {
baseBuiltinFunc
baseDurationBuiltinFunc
}

// eval evals a builtinSecToTimeSig.
// evalDuration evals SEC_TO_TIME(seconds).
// See https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_sec-to-time
func (b *builtinSecToTimeSig) eval(row []types.Datum) (d types.Datum, err error) {
args, err := b.evalArgs(row)
if err != nil {
return d, errors.Trace(err)
}
if args[0].IsNull() {
return
func (b *builtinSecToTimeSig) evalDuration(row []types.Datum) (types.Duration, bool, error) {
ctx := b.getCtx().GetSessionVars().StmtCtx
secondsFloat, isNull, err := b.args[0].EvalReal(row, ctx)
if isNull || err != nil {
return types.Duration{}, isNull, errors.Trace(err)
}

var (
hour int64
minute int64
Expand All @@ -2831,16 +2847,6 @@ func (b *builtinSecToTimeSig) eval(row []types.Datum) (d types.Datum, err error)
negative string
)

sc := b.ctx.GetSessionVars().StmtCtx
secondsFloat, err := args[0].ToFloat64(sc)
if err != nil {
if args[0].Kind() == types.KindString && types.ErrTruncated.Equal(err) {
secondsFloat = float64(0)
} else {
return d, errors.Trace(err)
}
}

if secondsFloat < 0 {
negative = "-"
secondsFloat = math.Abs(secondsFloat)
Expand All @@ -2860,22 +2866,11 @@ func (b *builtinSecToTimeSig) eval(row []types.Datum) (d types.Datum, err error)
secondDemical = float64(second) + demical

var dur types.Duration
fsp := types.MaxFsp
if args[0].Kind() != types.KindString {
sec, _ := args[0].ToString()
secs := strings.Split(sec, ".")
if len(secs) <= 1 {
fsp = 0
} else if len(secs[1]) < fsp {
fsp = len(secs[1])
}
}
dur, err = types.ParseDuration(fmt.Sprintf("%s%02d:%02d:%v", negative, hour, minute, secondDemical), fsp)
dur, err = types.ParseDuration(fmt.Sprintf("%s%02d:%02d:%v", negative, hour, minute, secondDemical), b.tp.Decimal)
if err != nil {
return d, errors.Trace(err)
return types.Duration{}, false, errors.Trace(err)
}
d.SetMysqlDuration(dur)
return
return dur, false, nil
}

type subTimeFunctionClass struct {
Expand Down Expand Up @@ -3012,35 +3007,31 @@ func (c *timeToSecFunctionClass) getFunction(ctx context.Context, args []Express
if err := c.verifyArgs(args); err != nil {
return nil, errors.Trace(err)
}
sig := &builtinTimeToSecSig{newBaseBuiltinFunc(args, ctx)}
bf := newBaseBuiltinFuncWithTp(args, ctx, tpInt, tpDuration)
bf.tp.Flen = 10
sig := &builtinTimeToSecSig{baseIntBuiltinFunc{bf}}
return sig.setSelf(sig), nil
}

type builtinTimeToSecSig struct {
baseBuiltinFunc
baseIntBuiltinFunc
}

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

d, err = convertToDuration(b.ctx.GetSessionVars().StmtCtx, args[0], 0)
if err != nil || d.IsNull() {
return d, errors.Trace(err)
func (b *builtinTimeToSecSig) evalInt(row []types.Datum) (int64, bool, error) {
ctx := b.getCtx().GetSessionVars().StmtCtx
duration, isNull, err := b.args[0].EvalDuration(row, ctx)
if isNull || err != nil {
return 0, isNull, errors.Trace(err)
}

t := d.GetMysqlDuration()
// TODO: select TIME_TO_SEC('-2:-2:-2') not handle well.
if t.Compare(types.ZeroDuration) < 0 {
d.SetInt64(int64(-1 * (t.Hour()*3600 + t.Minute()*60 + t.Second())))
var sign int
if duration.Duration >= 0 {
sign = 1
} else {
d.SetInt64(int64(t.Hour()*3600 + t.Minute()*60 + t.Second()))
sign = -1
}
return
return int64(sign * (duration.Hour()*3600 + duration.Minute()*60 + duration.Second())), false, nil
}

type timestampAddFunctionClass struct {
Expand Down
102 changes: 66 additions & 36 deletions expression/builtin_time_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1931,66 +1931,96 @@ func (s *testEvaluatorSuite) TestTimeFormat(c *C) {
}

func (s *testEvaluatorSuite) TestTimeToSec(c *C) {
fc := funcs[ast.TimeToSec]

// test nil
nilDatum := types.NewDatum(nil)
f, err := fc.getFunction(s.ctx, datumsToConstants([]types.Datum{nilDatum}))
c.Assert(err, IsNil)
c.Assert(f.canBeFolded(), IsTrue)
d, err := f.eval(nil)
c.Assert(err, IsNil)
c.Assert(d.Kind(), Equals, types.KindNull)

// TODO: Some test cases are commented out due to #4340, #4341.
tests := []struct {
t string
input types.Datum
expect int64
}{
{"22:23:00", 80580},
{"00:39:38", 2378},
{"23:00", 82800},
{"00:00", 0},
{"00:00:00", 0},
{"23:59:59", 86399},
{"1:0", 3600},
{"1:00", 3600},
{"1:0:0", 3600},
{"-02:00", -7200},
{types.NewStringDatum("22:23:00"), 80580},
{types.NewStringDatum("00:39:38"), 2378},
{types.NewStringDatum("23:00"), 82800},
{types.NewStringDatum("00:00"), 0},
{types.NewStringDatum("00:00:00"), 0},
{types.NewStringDatum("23:59:59"), 86399},
{types.NewStringDatum("1:0"), 3600},
{types.NewStringDatum("1:00"), 3600},
{types.NewStringDatum("1:0:0"), 3600},
{types.NewStringDatum("-02:00"), -7200},
{types.NewStringDatum("-02:00:05"), -7205},
{types.NewStringDatum("020005"), 7205},
// {types.NewStringDatum("20171222020005"), 7205},
// {types.NewIntDatum(020005), 7205},
// {types.NewIntDatum(20171222020005), 7205},
// {types.NewIntDatum(171222020005), 7205},
}
fc := funcs[ast.TimeToSec]
for _, test := range tests {
arg := types.NewStringDatum(test.t)
f, err := fc.getFunction(s.ctx, datumsToConstants([]types.Datum{arg}))
c.Assert(err, IsNil)
expr := datumsToConstants([]types.Datum{test.input})
f, err := fc.getFunction(s.ctx, expr)
c.Assert(err, IsNil, Commentf("%+v", test))
c.Assert(f.canBeFolded(), IsTrue, Commentf("%+v", test))
result, err := f.eval(nil)
c.Assert(err, IsNil)
c.Assert(result.GetInt64(), Equals, test.expect)
c.Assert(err, IsNil, Commentf("%+v", test))
c.Assert(result.GetInt64(), Equals, test.expect, Commentf("%+v", test))
}
}

func (s *testEvaluatorSuite) TestSecToTime(c *C) {
// Test cases from https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_sec-to-time
stmtCtx := s.ctx.GetSessionVars().StmtCtx
origin := stmtCtx.IgnoreTruncate
stmtCtx.IgnoreTruncate = true
defer func() {
stmtCtx.IgnoreTruncate = origin
}()

fc := funcs[ast.SecToTime]
//test nil

// test nil
nilDatum := types.NewDatum(nil)
f, err := fc.getFunction(s.ctx, datumsToConstants([]types.Datum{nilDatum}))
c.Assert(err, IsNil)
c.Assert(f.canBeFolded(), IsTrue)
d, err := f.eval(nil)
c.Assert(err, IsNil)
c.Assert(d.Kind(), Equals, types.KindNull)

tests := []struct {
param interface{}
expect string
inputDecimal int
input types.Datum
expect string
}{
{2378, "00:39:38"},
{3864000, "838:59:59"},
{-3864000, "-838:59:59"},
{86401.4, "24:00:01.4"},
{-86401.4, "-24:00:01.4"},
{86401.54321, "24:00:01.54321"},
{"123.4", "00:02:03.400000"},
{"123.4567891", "00:02:03.456789"},
{"123", "00:02:03.000000"},
{"abc", "00:00:00.000000"},
{0, types.NewIntDatum(2378), "00:39:38"},
{0, types.NewIntDatum(3864000), "838:59:59"},
{0, types.NewIntDatum(-3864000), "-838:59:59"},
{1, types.NewFloat64Datum(86401.4), "24:00:01.4"},
{1, types.NewFloat64Datum(-86401.4), "-24:00:01.4"},
{5, types.NewFloat64Datum(86401.54321), "24:00:01.54321"},
{-1, types.NewFloat64Datum(86401.54321), "24:00:01.543210"},
{0, types.NewStringDatum("123.4"), "00:02:03.400000"},
{0, types.NewStringDatum("123.4567891"), "00:02:03.456789"},
{0, types.NewStringDatum("123"), "00:02:03.000000"},
{0, types.NewStringDatum("abc"), "00:00:00.000000"},
}
for _, test := range tests {
t := []types.Datum{types.NewDatum(test.param)}
f, err := fc.getFunction(s.ctx, datumsToConstants(t))
c.Assert(err, IsNil)
expr := datumsToConstants([]types.Datum{test.input})
expr[0].GetType().Decimal = test.inputDecimal
f, err := fc.getFunction(s.ctx, expr)
c.Assert(err, IsNil, Commentf("%+v", test))
c.Assert(f.canBeFolded(), IsTrue, Commentf("%+v", test))
d, err := f.eval(nil)
c.Assert(err, IsNil)
c.Assert(err, IsNil, Commentf("%+v", test))
result, _ := d.ToString()
c.Assert(result, Equals, test.expect)
c.Assert(result, Equals, test.expect, Commentf("%+v", test))
}
}

Expand Down
27 changes: 24 additions & 3 deletions expression/integration_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1323,9 +1323,8 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) {
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(15111310201912e-2);")
result.Check(testkit.Rows("1447410019.120000"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(151113102019.12);")
result.Check(testkit.Rows("1447410019.12"))
result = tk.MustQuery("SELECT UNIX_TIMESTAMP(151113102019.1234567);")
Expand Down Expand Up @@ -1402,6 +1401,28 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) {
result.Check(testkit.Rows("Friday <nil> <nil> <nil>"))
tk.MustQuery("show warnings").Check(testutil.RowsWithSep("|", "Warning|1105|invalid time format", "Warning|1105|invalid time format", "Warning|1105|invalid time format"))

// for sec_to_time
result = tk.MustQuery("select sec_to_time(NULL)")
result.Check(testkit.Rows("<nil>"))
result = tk.MustQuery("select sec_to_time(2378), sec_to_time(3864000), sec_to_time(-3864000)")
result.Check(testkit.Rows("00:39:38 838:59:59 -838:59:59"))
result = tk.MustQuery("select sec_to_time(86401.4), sec_to_time(-86401.4), sec_to_time(864014e-1), sec_to_time(-864014e-1), sec_to_time('86401.4'), sec_to_time('-86401.4')")
result.Check(testkit.Rows("24:00:01.4 -24:00:01.4 24:00:01.400000 -24:00:01.400000 24:00:01.400000 -24:00:01.400000"))
result = tk.MustQuery("select sec_to_time(86401.54321), sec_to_time(86401.543212345)")
result.Check(testkit.Rows("24:00:01.54321 24:00:01.543212"))
result = tk.MustQuery("select sec_to_time('123.4'), sec_to_time('123.4567891'), sec_to_time('123')")
result.Check(testkit.Rows("00:02:03.400000 00:02:03.456789 00:02:03.000000"))

// for time_to_sec
result = tk.MustQuery("select time_to_sec(NULL)")
result.Check(testkit.Rows("<nil>"))
result = tk.MustQuery("select time_to_sec('22:23:00'), time_to_sec('00:39:38'), time_to_sec('23:00'), time_to_sec('00:00'), time_to_sec('00:00:00'), time_to_sec('23:59:59')")
result.Check(testkit.Rows("80580 2378 82800 0 0 86399"))
result = tk.MustQuery("select time_to_sec('1:0'), time_to_sec('1:00'), time_to_sec('1:0:0'), time_to_sec('-02:00'), time_to_sec('-02:00:05'), time_to_sec('020005')")
result.Check(testkit.Rows("3600 3600 3600 -7200 -7205 7205"))
result = tk.MustQuery("select time_to_sec('20171222020005'), time_to_sec(020005), time_to_sec(20171222020005), time_to_sec(171222020005)")
result.Check(testkit.Rows("7205 7205 7205 7205"))

// for str_to_date
result = tk.MustQuery("select str_to_date('01-01-2017', '%d-%m-%Y'), str_to_date('59:20:12 01-01-2017', '%s:%i:%H %d-%m-%Y'), str_to_date('59:20:12', '%s:%i:%H')")
result.Check(testkit.Rows("2017-01-01 2017-01-01 12:20:59 12:20:59"))
Expand Down
4 changes: 2 additions & 2 deletions expression/typeinferer.go
Original file line number Diff line number Diff line change
Expand Up @@ -335,7 +335,7 @@ func (v *typeInferrer) handleFuncCallExpr(x *ast.FuncCallExpr) {
ast.DayOfWeek, ast.DayOfMonth, ast.DayOfYear, ast.Weekday, ast.WeekOfYear, ast.YearWeek, ast.DateDiff,
ast.FoundRows, ast.Length, ast.ASCII, ast.Extract, ast.Locate, ast.UnixTimestamp, ast.Quarter, ast.IsIPv4, ast.ToDays,
ast.ToSeconds, ast.Strcmp, ast.IsNull, ast.BitLength, ast.CharLength, ast.CRC32, ast.TimestampDiff,
ast.Sign, ast.IsIPv6, ast.Ord, ast.Instr, ast.BitCount, ast.TimeToSec, ast.FindInSet, ast.Field,
ast.Sign, ast.IsIPv6, ast.Ord, ast.Instr, ast.BitCount, ast.FindInSet, ast.Field,
ast.GetLock, ast.ReleaseLock, ast.Interval, ast.Position, ast.PeriodAdd, ast.PeriodDiff, ast.IsIPv4Mapped, ast.IsIPv4Compat, ast.UncompressedLength:
tp = types.NewFieldType(mysql.TypeLonglong)
case ast.ConnectionID, ast.InetAton:
Expand All @@ -362,7 +362,7 @@ func (v *typeInferrer) handleFuncCallExpr(x *ast.FuncCallExpr) {
tp = types.NewFieldType(mysql.TypeVarString)
}
tp.Charset, tp.Collate = types.DefaultCharsetForType(tp.Tp)
case ast.Curtime, ast.CurrentTime, ast.TimeDiff, ast.MakeTime, ast.SecToTime, ast.UTCTime, ast.Time:
case ast.Curtime, ast.CurrentTime, ast.TimeDiff, ast.MakeTime, ast.UTCTime, ast.Time:
tp = types.NewFieldType(mysql.TypeDuration)
tp.Decimal = v.getFsp(x)
case ast.Curdate, ast.CurrentDate, ast.Date, ast.FromDays, ast.MakeDate, ast.LastDay:
Expand Down
Loading

0 comments on commit 0513ab3

Please sign in to comment.