diff --git a/expression/date_add.go b/expression/date_add.go deleted file mode 100644 index 95cd99a0deed5..0000000000000 --- a/expression/date_add.go +++ /dev/null @@ -1,104 +0,0 @@ -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package expression - -import ( - "fmt" - "strings" - - "github.com/juju/errors" - "github.com/pingcap/tidb/context" - "github.com/pingcap/tidb/mysql" - "github.com/pingcap/tidb/util/types" -) - -// DateAdd is for time date_add function. -// See: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_date-add -type DateAdd struct { - Unit string - Date Expression - Interval Expression -} - -// Clone implements the Expression Clone interface. -func (da *DateAdd) Clone() Expression { - n := *da - return &n -} - -// Eval implements the Expression Eval interface. -func (da *DateAdd) Eval(ctx context.Context, args map[interface{}]interface{}) (interface{}, error) { - dv, err := da.Date.Eval(ctx, args) - if dv == nil || err != nil { - return nil, errors.Trace(err) - } - - sv, err := types.ToString(dv) - if err != nil { - return nil, errors.Trace(err) - } - - f := types.NewFieldType(mysql.TypeDatetime) - f.Decimal = mysql.MaxFsp - - dv, err = types.Convert(sv, f) - if dv == nil || err != nil { - return nil, errors.Trace(err) - } - - t, ok := dv.(mysql.Time) - if !ok { - return nil, errors.Errorf("need time type, but got %T", dv) - } - - iv, err := da.Interval.Eval(ctx, args) - if iv == nil || err != nil { - return nil, errors.Trace(err) - } - - format, err := types.ToString(iv) - if err != nil { - return nil, errors.Trace(err) - } - - years, months, days, durations, err := mysql.ExtractTimeValue(da.Unit, strings.TrimSpace(format)) - if err != nil { - return nil, errors.Trace(err) - } - - t.Time = t.Time.Add(durations) - t.Time = t.Time.AddDate(int(years), int(months), int(days)) - - // "2011-11-11 10:10:20.000000" outputs "2011-11-11 10:10:20". - if t.Time.Nanosecond() == 0 { - t.Fsp = 0 - } - - return t, nil -} - -// IsStatic implements the Expression IsStatic interface. -func (da *DateAdd) IsStatic() bool { - return da.Date.IsStatic() && da.Interval.IsStatic() -} - -// String implements the Expression String interface. -func (da *DateAdd) String() string { - return fmt.Sprintf("DATE_ADD(%s, INTERVAL %s %s)", da.Date, da.Interval, strings.ToUpper(da.Unit)) -} - -// Accept implements the Visitor Accept interface. -func (da *DateAdd) Accept(v Visitor) (Expression, error) { - return v.VisitDateAdd(da) -} diff --git a/expression/date_add_test.go b/expression/date_add_test.go deleted file mode 100644 index 6b612dce40c3e..0000000000000 --- a/expression/date_add_test.go +++ /dev/null @@ -1,144 +0,0 @@ -// Copyright 2015 PingCAP, Inc. -// -// Licensed under the Apache License, Version 2.0 (the "License"); -// you may not use this file except in compliance with the License. -// You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, software -// distributed under the License is distributed on an "AS IS" BASIS, -// See the License for the specific language governing permissions and -// limitations under the License. - -package expression - -import ( - . "github.com/pingcap/check" - "github.com/pingcap/tidb/mysql" -) - -var _ = Suite(&testDateAddSuite{}) - -type testDateAddSuite struct { -} - -func (t *testDateAddSuite) TestDateAdd(c *C) { - input := "2011-11-11 10:10:10" - e := &DateAdd{ - Unit: "DAY", - Date: Value{Val: input}, - Interval: Value{Val: "1"}, - } - c.Assert(e.String(), Equals, `DATE_ADD("2011-11-11 10:10:10", INTERVAL "1" DAY)`) - c.Assert(e.Clone(), NotNil) - c.Assert(e.IsStatic(), IsTrue) - - _, err := e.Eval(nil, nil) - c.Assert(err, IsNil) - - // Test null. - e = &DateAdd{ - Unit: "DAY", - Date: Value{Val: nil}, - Interval: Value{Val: "1"}, - } - - v, err := e.Eval(nil, nil) - c.Assert(err, IsNil) - c.Assert(v, IsNil) - - e = &DateAdd{ - Unit: "DAY", - Date: Value{Val: input}, - Interval: Value{Val: nil}, - } - - v, err = e.Eval(nil, nil) - c.Assert(err, IsNil) - c.Assert(v, IsNil) - - // Test eval. - tbl := []struct { - Unit string - Interval interface{} - Expect string - }{ - {"MICROSECOND", "1000", "2011-11-11 10:10:10.001000"}, - {"MICROSECOND", 1000, "2011-11-11 10:10:10.001000"}, - {"SECOND", "10", "2011-11-11 10:10:20"}, - {"MINUTE", "10", "2011-11-11 10:20:10"}, - {"HOUR", "10", "2011-11-11 20:10:10"}, - {"DAY", "11", "2011-11-22 10:10:10"}, - {"WEEK", "2", "2011-11-25 10:10:10"}, - {"MONTH", "2", "2012-01-11 10:10:10"}, - {"QUARTER", "4", "2012-11-11 10:10:10"}, - {"YEAR", "2", "2013-11-11 10:10:10"}, - {"SECOND_MICROSECOND", "10.00100000", "2011-11-11 10:10:20.100000"}, - {"SECOND_MICROSECOND", "10.0010000000", "2011-11-11 10:10:30"}, - {"SECOND_MICROSECOND", "10.0010000010", "2011-11-11 10:10:30.000010"}, - {"MINUTE_MICROSECOND", "10:10.100", "2011-11-11 10:20:20.100000"}, - {"MINUTE_SECOND", "10:10", "2011-11-11 10:20:20"}, - {"HOUR_MICROSECOND", "10:10:10.100", "2011-11-11 20:20:20.100000"}, - {"HOUR_SECOND", "10:10:10", "2011-11-11 20:20:20"}, - {"HOUR_MINUTE", "10:10", "2011-11-11 20:20:10"}, - {"DAY_MICROSECOND", "11 10:10:10.100", "2011-11-22 20:20:20.100000"}, - {"DAY_SECOND", "11 10:10:10", "2011-11-22 20:20:20"}, - {"DAY_MINUTE", "11 10:10", "2011-11-22 20:20:10"}, - {"DAY_HOUR", "11 10", "2011-11-22 20:10:10"}, - {"YEAR_MONTH", "11-1", "2022-12-11 10:10:10"}, - {"YEAR_MONTH", "11-11", "2023-10-11 10:10:10"}, - } - - for _, t := range tbl { - e := &DateAdd{ - Unit: t.Unit, - Date: Value{Val: input}, - Interval: Value{Val: t.Interval}, - } - - v, err := e.Eval(nil, nil) - c.Assert(err, IsNil) - - value, ok := v.(mysql.Time) - c.Assert(ok, IsTrue) - c.Assert(value.String(), Equals, t.Expect) - } - - // Test error. - errInput := "20111111 10:10:10" - errTbl := []struct { - Unit string - Interval interface{} - }{ - {"MICROSECOND", "abc1000"}, - {"MICROSECOND", ""}, - {"SECOND_MICROSECOND", "10"}, - {"MINUTE_MICROSECOND", "10.0000"}, - {"MINUTE_MICROSECOND", "10:10:10.0000"}, - - // MySQL support, but tidb not. - {"HOUR_MICROSECOND", "10:10.0000"}, - {"YEAR_MONTH", "10 1"}, - } - - for _, t := range errTbl { - e := &DateAdd{ - Unit: t.Unit, - Date: Value{Val: input}, - Interval: Value{Val: t.Interval}, - } - - _, err := e.Eval(nil, nil) - c.Assert(err, NotNil) - - e = &DateAdd{ - Unit: t.Unit, - Date: Value{Val: errInput}, - Interval: Value{Val: t.Interval}, - } - - v, err := e.Eval(nil, nil) - c.Assert(err, NotNil, Commentf("%s", v)) - } -} diff --git a/expression/date_arith.go b/expression/date_arith.go new file mode 100644 index 0000000000000..0a9a6c0de34ed --- /dev/null +++ b/expression/date_arith.go @@ -0,0 +1,139 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package expression + +import ( + "fmt" + "strings" + "time" + + "github.com/juju/errors" + "github.com/pingcap/tidb/context" + "github.com/pingcap/tidb/mysql" + "github.com/pingcap/tidb/util/types" +) + +// DateArithType is type for DateArith option. +type DateArithType byte + +const ( + // DateAdd is to run date_add function option. + // See: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_date-add + DateAdd DateArithType = iota + 1 + // DateSub is to run date_sub function option. + // See: https://dev.mysql.com/doc/refman/5.7/en/date-and-time-functions.html#function_date-sub + DateSub +) + +// DateArith is used for dealing with addition and substraction of time. +type DateArith struct { + Op DateArithType + Unit string + Date Expression + Interval Expression +} + +func (da *DateArith) isAdd() bool { + if da.Op == DateAdd { + return true + } + + return false +} + +// Clone implements the Expression Clone interface. +func (da *DateArith) Clone() Expression { + n := *da + return &n +} + +// IsStatic implements the Expression IsStatic interface. +func (da *DateArith) IsStatic() bool { + return da.Date.IsStatic() && da.Interval.IsStatic() +} + +// Accept implements the Visitor Accept interface. +func (da *DateArith) Accept(v Visitor) (Expression, error) { + return v.VisitDateArith(da) +} + +// String implements the Expression String interface. +func (da *DateArith) String() string { + var str string + if da.isAdd() { + str = "DATE_ADD" + } else { + str = "DATE_SUB" + } + + return fmt.Sprintf("%s(%s, INTERVAL %s %s)", str, da.Date, da.Interval, strings.ToUpper(da.Unit)) +} + +// Eval implements the Expression Eval interface. +func (da *DateArith) Eval(ctx context.Context, args map[interface{}]interface{}) (interface{}, error) { + t, years, months, days, durations, err := da.evalArgs(ctx, args) + if t.IsZero() || err != nil { + return nil, errors.Trace(err) + } + + if !da.isAdd() { + years, months, days, durations = -years, -months, -days, -durations + } + t.Time = t.Time.Add(durations) + t.Time = t.Time.AddDate(int(years), int(months), int(days)) + + // "2011-11-11 10:10:20.000000" outputs "2011-11-11 10:10:20". + if t.Time.Nanosecond() == 0 { + t.Fsp = 0 + } + + return t, nil +} + +func (da *DateArith) evalArgs(ctx context.Context, args map[interface{}]interface{}) ( + mysql.Time, int64, int64, int64, time.Duration, error) { + dVal, err := da.Date.Eval(ctx, args) + if dVal == nil || err != nil { + return mysql.ZeroTimestamp, 0, 0, 0, 0, errors.Trace(err) + } + dValStr, err := types.ToString(dVal) + if err != nil { + return mysql.ZeroTimestamp, 0, 0, 0, 0, errors.Trace(err) + } + f := types.NewFieldType(mysql.TypeDatetime) + f.Decimal = mysql.MaxFsp + dVal, err = types.Convert(dValStr, f) + if dVal == nil || err != nil { + return mysql.ZeroTimestamp, 0, 0, 0, 0, errors.Trace(err) + } + t, ok := dVal.(mysql.Time) + if !ok { + return mysql.ZeroTimestamp, 0, 0, 0, 0, errors.Errorf("need time type, but got %T", dVal) + } + + iVal, err := da.Interval.Eval(ctx, args) + if iVal == nil || err != nil { + return mysql.ZeroTimestamp, 0, 0, 0, 0, errors.Trace(err) + } + iValStr, err := types.ToString(iVal) + if err != nil { + return mysql.ZeroTimestamp, 0, 0, 0, 0, errors.Trace(err) + } + years, months, days, durations, err := mysql.ExtractTimeValue(da.Unit, strings.TrimSpace(iValStr)) + if err != nil { + return mysql.ZeroTimestamp, 0, 0, 0, 0, errors.Trace(err) + } + + return t, years, months, days, durations, nil +} diff --git a/expression/date_arith_test.go b/expression/date_arith_test.go new file mode 100644 index 0000000000000..2775d7d914868 --- /dev/null +++ b/expression/date_arith_test.go @@ -0,0 +1,162 @@ +// Copyright 2015 PingCAP, Inc. +// +// Licensed under the Apache License, Version 2.0 (the "License"); +// you may not use this file except in compliance with the License. +// You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, software +// distributed under the License is distributed on an "AS IS" BASIS, +// See the License for the specific language governing permissions and +// limitations under the License. + +package expression + +import ( + . "github.com/pingcap/check" + "github.com/pingcap/tidb/mysql" +) + +var _ = Suite(&testDateArithSuite{}) + +type testDateArithSuite struct { +} + +func (t *testDateArithSuite) TestDateArith(c *C) { + input := "2011-11-11 10:10:10" + e := &DateArith{ + Op: DateAdd, + Unit: "DAY", + Date: Value{Val: input}, + Interval: Value{Val: "1"}, + } + c.Assert(e.String(), Equals, `DATE_ADD("2011-11-11 10:10:10", INTERVAL "1" DAY)`) + c.Assert(e.Clone(), NotNil) + c.Assert(e.IsStatic(), IsTrue) + _, err := e.Eval(nil, nil) + c.Assert(err, IsNil) + e.Op = DateSub + c.Assert(e.String(), Equals, `DATE_SUB("2011-11-11 10:10:10", INTERVAL "1" DAY)`) + + // Test null. + nullTbl := []struct { + Op DateArithType + Unit string + Date interface{} + Interval interface{} + }{ + {DateAdd, "DAY", nil, "1"}, + {DateAdd, "DAY", input, nil}, + } + for _, t := range nullTbl { + e := &DateArith{ + Op: t.Op, + Unit: t.Unit, + Date: Value{Val: t.Date}, + Interval: Value{Val: t.Interval}, + } + v, err := e.Eval(nil, nil) + c.Assert(err, IsNil) + c.Assert(v, IsNil) + e.Op = DateSub + v, err = e.Eval(nil, nil) + c.Assert(err, IsNil) + c.Assert(v, IsNil) + } + + // Test eval. + tbl := []struct { + Unit string + Interval interface{} + AddExpect string + SubExpect string + }{ + {"MICROSECOND", "1000", "2011-11-11 10:10:10.001000", "2011-11-11 10:10:09.999000"}, + {"MICROSECOND", 1000, "2011-11-11 10:10:10.001000", "2011-11-11 10:10:09.999000"}, + {"SECOND", "10", "2011-11-11 10:10:20", "2011-11-11 10:10:00"}, + {"MINUTE", "10", "2011-11-11 10:20:10", "2011-11-11 10:00:10"}, + {"HOUR", "10", "2011-11-11 20:10:10", "2011-11-11 00:10:10"}, + {"DAY", "11", "2011-11-22 10:10:10", "2011-10-31 10:10:10"}, + {"WEEK", "2", "2011-11-25 10:10:10", "2011-10-28 10:10:10"}, + {"MONTH", "2", "2012-01-11 10:10:10", "2011-09-11 10:10:10"}, + {"QUARTER", "4", "2012-11-11 10:10:10", "2010-11-11 10:10:10"}, + {"YEAR", "2", "2013-11-11 10:10:10", "2009-11-11 10:10:10"}, + {"SECOND_MICROSECOND", "10.00100000", "2011-11-11 10:10:20.100000", "2011-11-11 10:09:59.900000"}, + {"SECOND_MICROSECOND", "10.0010000000", "2011-11-11 10:10:30", "2011-11-11 10:09:50"}, + {"SECOND_MICROSECOND", "10.0010000010", "2011-11-11 10:10:30.000010", "2011-11-11 10:09:49.999990"}, + {"MINUTE_MICROSECOND", "10:10.100", "2011-11-11 10:20:20.100000", "2011-11-11 09:59:59.900000"}, + {"MINUTE_SECOND", "10:10", "2011-11-11 10:20:20", "2011-11-11 10:00:00"}, + {"HOUR_MICROSECOND", "10:10:10.100", "2011-11-11 20:20:20.100000", "2011-11-10 23:59:59.900000"}, + {"HOUR_SECOND", "10:10:10", "2011-11-11 20:20:20", "2011-11-11 00:00:00"}, + {"HOUR_MINUTE", "10:10", "2011-11-11 20:20:10", "2011-11-11 00:00:10"}, + {"DAY_MICROSECOND", "11 10:10:10.100", "2011-11-22 20:20:20.100000", "2011-10-30 23:59:59.900000"}, + {"DAY_SECOND", "11 10:10:10", "2011-11-22 20:20:20", "2011-10-31 00:00:00"}, + {"DAY_MINUTE", "11 10:10", "2011-11-22 20:20:10", "2011-10-31 00:00:10"}, + {"DAY_HOUR", "11 10", "2011-11-22 20:10:10", "2011-10-31 00:10:10"}, + {"YEAR_MONTH", "11-1", "2022-12-11 10:10:10", "2000-10-11 10:10:10"}, + {"YEAR_MONTH", "11-11", "2023-10-11 10:10:10", "1999-12-11 10:10:10"}, + } + for _, t := range tbl { + e := &DateArith{ + Op: DateAdd, + Unit: t.Unit, + Date: Value{Val: input}, + Interval: Value{Val: t.Interval}, + } + v, err := e.Eval(nil, nil) + c.Assert(err, IsNil) + value, ok := v.(mysql.Time) + c.Assert(ok, IsTrue) + c.Assert(value.String(), Equals, t.AddExpect) + + e.Op = DateSub + v, err = e.Eval(nil, nil) + c.Assert(err, IsNil) + value, ok = v.(mysql.Time) + c.Assert(ok, IsTrue) + c.Assert(value.String(), Equals, t.SubExpect) + } + + // Test error. + errInput := "20111111 10:10:10" + errTbl := []struct { + Unit string + Interval interface{} + }{ + {"MICROSECOND", "abc1000"}, + {"MICROSECOND", ""}, + {"SECOND_MICROSECOND", "10"}, + {"MINUTE_MICROSECOND", "10.0000"}, + {"MINUTE_MICROSECOND", "10:10:10.0000"}, + + // MySQL support, but tidb not. + {"HOUR_MICROSECOND", "10:10.0000"}, + {"YEAR_MONTH", "10 1"}, + } + for _, t := range errTbl { + e := &DateArith{ + Op: DateAdd, + Unit: t.Unit, + Date: Value{Val: input}, + Interval: Value{Val: t.Interval}, + } + _, err := e.Eval(nil, nil) + c.Assert(err, NotNil) + e.Date = Value{Val: errInput} + v, err := e.Eval(nil, nil) + c.Assert(err, NotNil, Commentf("%s", v)) + + e = &DateArith{ + Op: DateSub, + Unit: t.Unit, + Date: Value{Val: input}, + Interval: Value{Val: t.Interval}, + } + _, err = e.Eval(nil, nil) + c.Assert(err, NotNil) + e.Date = Value{Val: errInput} + v, err = e.Eval(nil, nil) + c.Assert(err, NotNil, Commentf("%s", v)) + } +} diff --git a/expression/visitor.go b/expression/visitor.go index 6ec16f765f5c3..6646d59db7f6c 100644 --- a/expression/visitor.go +++ b/expression/visitor.go @@ -107,8 +107,8 @@ type Visitor interface { // VisitFunctionTrim visits FunctionTrim expression. VisitFunctionTrim(v *FunctionTrim) (Expression, error) - // VisitDateAdd visits DateAdd expression. - VisitDateAdd(da *DateAdd) (Expression, error) + // VisitDateArith visits DateArith expression. + VisitDateArith(dc *DateArith) (Expression, error) } // BaseVisitor is the base implementation of Visitor. @@ -482,8 +482,8 @@ func (bv *BaseVisitor) VisitFunctionTrim(ss *FunctionTrim) (Expression, error) { return ss, nil } -// VisitDateAdd implements Visitor interface. -func (bv *BaseVisitor) VisitDateAdd(da *DateAdd) (Expression, error) { +// VisitDateArith implements Visitor interface. +func (bv *BaseVisitor) VisitDateArith(da *DateArith) (Expression, error) { var err error da.Date, err = da.Date.Accept(bv.V) if err != nil { diff --git a/parser/parser.y b/parser/parser.y index 2f938db348de3..d918b17ee175f 100644 --- a/parser/parser.y +++ b/parser/parser.y @@ -114,6 +114,7 @@ import ( database "DATABASE" databases "DATABASES" dateAdd "DATE_ADD" + dateSub "DATE_SUB" day "DAY" dayofmonth "DAYOFMONTH" dayofweek "DAYOFWEEK" @@ -1714,7 +1715,7 @@ UnReservedKeyword: | "NATIONAL" | "ROW" | "QUARTER" | "ESCAPE" | "GRANTS" NotKeywordToken: - "ABS" | "COALESCE" | "CONCAT" | "CONCAT_WS" | "COUNT" | "DAY" | "DATE_ADD" | "DAYOFMONTH" | "DAYOFWEEK" | "DAYOFYEAR" | "FOUND_ROWS" | "GROUP_CONCAT" + "ABS" | "COALESCE" | "CONCAT" | "CONCAT_WS" | "COUNT" | "DAY" | "DATE_ADD" | "DATE_SUB" | "DAYOFMONTH" | "DAYOFWEEK" | "DAYOFYEAR" | "FOUND_ROWS" | "GROUP_CONCAT" | "HOUR" | "IFNULL" | "LENGTH" | "LOCATE" | "MAX" | "MICROSECOND" | "MIN" | "MINUTE" | "NULLIF" | "MONTH" | "NOW" | "RAND" | "SECOND" | "SQL_CALC_FOUND_ROWS" | "SUBSTRING" %prec lowerThanLeftParen | "SUBSTRING_INDEX" | "SUM" | "TRIM" | "WEEKDAY" | "WEEKOFYEAR" | "YEARWEEK" @@ -2306,12 +2307,22 @@ FunctionCallNonKeyword: } | "DATE_ADD" '(' Expression ',' "INTERVAL" Expression TimeUnit ')' { - $$ = &expression.DateAdd{ + $$ = &expression.DateArith{ + Op:expression.DateAdd, Unit: $7.(string), Date: $3.(expression.Expression), Interval: $6.(expression.Expression), } } +| "DATE_SUB" '(' Expression ',' "INTERVAL" Expression TimeUnit ')' + { + $$ = &expression.DateArith{ + Op:expression.DateSub, + Unit: $7.(string), + Date: $3.(expression.Expression), + Interval: $6.(expression.Expression), + } + } | "EXTRACT" '(' TimeUnit "FROM" Expression ')' { $$ = &expression.Extract{ diff --git a/parser/parser_test.go b/parser/parser_test.go index f0fa85c6cb856..8f5c0d10ee944 100644 --- a/parser/parser_test.go +++ b/parser/parser_test.go @@ -441,6 +441,28 @@ func (s *testParserSuite) TestBuiltin(c *C) { {`select date_add("2011-11-11 10:10:10.123456", interval "11 10:10" day_minute)`, true}, {`select date_add("2011-11-11 10:10:10.123456", interval "11 10" day_hour)`, true}, {`select date_add("2011-11-11 10:10:10.123456", interval "11-11" year_month)`, true}, + + // For date_sub + {`select date_sub("2011-11-11 10:10:10.123456", interval 10 microsecond)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval 10 second)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval 10 minute)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval 10 hour)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval 10 day)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval 1 week)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval 1 month)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval 1 quarter)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval 1 year)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "10.10" second_microsecond)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "10:10.10" minute_microsecond)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "10:10" minute_second)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "10:10:10.10" hour_microsecond)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "10:10:10" hour_second)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "10:10" hour_minute)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "11 10:10:10.10" day_microsecond)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "11 10:10:10" day_second)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "11 10:10" day_minute)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "11 10" day_hour)`, true}, + {`select date_sub("2011-11-11 10:10:10.123456", interval "11-11" year_month)`, true}, } s.RunTest(c, table) } diff --git a/parser/scanner.l b/parser/scanner.l index 2b6fcf051f6c8..c1441b4650ebc 100644 --- a/parser/scanner.l +++ b/parser/scanner.l @@ -299,6 +299,7 @@ current_user {c}{u}{r}{r}{e}{n}{t}_{u}{s}{e}{r} database {d}{a}{t}{a}{b}{a}{s}{e} databases {d}{a}{t}{a}{b}{a}{s}{e}{s} date_add {d}{a}{t}{e}_{a}{d}{d} +date_sub {d}{a}{t}{e}_{s}{u}{b} day {d}{a}{y} dayofweek {d}{a}{y}{o}{f}{w}{e}{e}{k} dayofmonth {d}{a}{y}{o}{f}{m}{o}{n}{t}{h} @@ -650,6 +651,8 @@ year_month {y}{e}{a}{r}_{m}{o}{n}{t}{h} {databases} return databases {date_add} lval.item = string(l.val) return dateAdd +{date_sub} lval.item = string(l.val) + return dateSub {day} lval.item = string(l.val) return day {dayofweek} lval.item = string(l.val)