From 0513ab378d45de9b2dfa2160287d3de02c0284ae Mon Sep 17 00:00:00 2001 From: Wenxuan Shi Date: Mon, 4 Sep 2017 22:36:04 -0500 Subject: [PATCH] expression: rewrite builtin function: TIME_TO_SEC, SEC_TO_TIME (#4342) --- expression/builtin_time.go | 99 ++++++++++++++----------------- expression/builtin_time_test.go | 102 +++++++++++++++++++++----------- expression/integration_test.go | 27 ++++++++- expression/typeinferer.go | 4 +- plan/typeinfer_test.go | 47 +++++++++++++-- 5 files changed, 180 insertions(+), 99 deletions(-) diff --git a/expression/builtin_time.go b/expression/builtin_time.go index bffe48d055e8f..fbee03ab4a9a9 100644 --- a/expression/builtin_time.go +++ b/expression/builtin_time.go @@ -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 @@ -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) @@ -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 { @@ -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 { diff --git a/expression/builtin_time_test.go b/expression/builtin_time_test.go index 3dbd1dacb6a8a..26d75a7c397a5 100644 --- a/expression/builtin_time_test.go +++ b/expression/builtin_time_test.go @@ -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)) } } diff --git a/expression/integration_test.go b/expression/integration_test.go index a71019c5ccf74..7c4230d1b8815 100644 --- a/expression/integration_test.go +++ b/expression/integration_test.go @@ -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);") @@ -1402,6 +1401,28 @@ func (s *testIntegrationSuite) TestTimeBuiltin(c *C) { result.Check(testkit.Rows("Friday ")) 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("")) + 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("")) + 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")) diff --git a/expression/typeinferer.go b/expression/typeinferer.go index c1f4c334e4730..70f7d99af53a9 100644 --- a/expression/typeinferer.go +++ b/expression/typeinferer.go @@ -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: @@ -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: diff --git a/plan/typeinfer_test.go b/plan/typeinfer_test.go index b953487e0de09..173c2fef411dc 100644 --- a/plan/typeinfer_test.go +++ b/plan/typeinfer_test.go @@ -1099,9 +1099,8 @@ 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_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}, @@ -1111,7 +1110,7 @@ func (s *testPlanSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"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_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}, @@ -1594,6 +1593,46 @@ func (s *testPlanSuite) createTestCase4TimeFuncs() []typeInferTestCase { {"quarter(c_set )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, {"quarter(c_enum )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 1, 0}, + {"sec_to_time(c_int_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"sec_to_time(c_bigint_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"sec_to_time(c_float_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + {"sec_to_time(c_double_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + {"sec_to_time(c_decimal )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 14, 3}, + {"sec_to_time(c_decimal_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"sec_to_time(c_datetime )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 13, 2}, + {"sec_to_time(c_time )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 14, 3}, + {"sec_to_time(c_time_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"sec_to_time(c_timestamp )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 15, 4}, + {"sec_to_time(c_timestamp_d)", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"sec_to_time(c_char )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + {"sec_to_time(c_varchar )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + {"sec_to_time(c_text_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + {"sec_to_time(c_binary )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + {"sec_to_time(c_varbinary )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + {"sec_to_time(c_blob_d )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + {"sec_to_time(c_set )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + {"sec_to_time(c_enum )", mysql.TypeDuration, charset.CharsetBin, mysql.BinaryFlag, 17, 6}, + + {"time_to_sec(c_int_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_bigint_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_float_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_double_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_decimal )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_decimal_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_datetime )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_time )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_time_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_timestamp )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_timestamp_d)", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_char )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_varchar )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_text_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_binary )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_varbinary )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_blob_d )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_set )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"time_to_sec(c_enum )", mysql.TypeLonglong, charset.CharsetBin, mysql.BinaryFlag, 10, 0}, + {"str_to_date(c_varchar, '%Y:%m:%d')", mysql.TypeDate, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDateWidth, types.MinFsp}, {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthNoFsp, types.MinFsp}, {"str_to_date(c_varchar, '%Y:%m:%d %H:%i:%s.%f')", mysql.TypeDatetime, charset.CharsetBin, mysql.BinaryFlag, mysql.MaxDatetimeWidthWithFsp, types.MaxFsp},