Add DATE and TIMESTAMP operators (#9691)

* DATE and TIMESTAMP operators
This commit is contained in:
Artem Zuikov 2020-03-18 00:56:47 +03:00 committed by GitHub
parent c19c954f1e
commit dbc08d7644
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 157 additions and 31 deletions

View File

@ -595,6 +595,69 @@ bool ParserNullityChecking::parseImpl(Pos & pos, ASTPtr & node, Expected & expec
return true;
}
bool ParserDateOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto begin = pos;
/// If no DATE keyword, go to the nested parser.
if (!ParserKeyword("DATE").ignore(pos, expected))
return next_parser.parse(pos, node, expected);
ASTPtr expr;
if (!ParserStringLiteral().parse(pos, expr, expected))
{
pos = begin;
return next_parser.parse(pos, node, expected);
}
/// the function corresponding to the operator
auto function = std::make_shared<ASTFunction>();
/// function arguments
auto exp_list = std::make_shared<ASTExpressionList>();
/// the first argument of the function is the previous element, the second is the next one
function->name = "toDate";
function->arguments = exp_list;
function->children.push_back(exp_list);
exp_list->children.push_back(expr);
node = function;
return true;
}
bool ParserTimestampOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
auto begin = pos;
/// If no TIMESTAMP keyword, go to the nested parser.
if (!ParserKeyword("TIMESTAMP").ignore(pos, expected))
return next_parser.parse(pos, node, expected);
ASTPtr expr;
if (!ParserStringLiteral().parse(pos, expr, expected))
{
pos = begin;
return next_parser.parse(pos, node, expected);
}
/// the function corresponding to the operator
auto function = std::make_shared<ASTFunction>();
/// function arguments
auto exp_list = std::make_shared<ASTExpressionList>();
/// the first argument of the function is the previous element, the second is the next one
function->name = "toDateTime";
function->arguments = exp_list;
function->children.push_back(exp_list);
exp_list->children.push_back(expr);
node = function;
return true;
}
bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{

View File

@ -159,7 +159,7 @@ private:
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserUnaryMinusExpression>()};
protected:
const char * getName() const override{ return "multiplicative expression"; }
const char * getName() const override { return "multiplicative expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
@ -167,18 +167,36 @@ protected:
}
};
/// DATE operator. "DATE '2001-01-01'" would be parsed as "toDate('2001-01-01')".
class ParserDateOperatorExpression : public IParserBase
{
protected:
ParserMultiplicativeExpression next_parser;
const char * getName() const override { return "DATE operator expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// TIMESTAMP operator. "TIMESTAMP '2001-01-01 12:34:56'" would be parsed as "toDateTime('2001-01-01 12:34:56')".
class ParserTimestampOperatorExpression : public IParserBase
{
protected:
ParserDateOperatorExpression next_parser;
const char * getName() const override { return "TIMESTAMP operator expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// Optional conversion to INTERVAL data type. Example: "INTERVAL x SECOND" parsed as "toIntervalSecond(x)".
class ParserIntervalOperatorExpression : public IParserBase
{
protected:
ParserMultiplicativeExpression next_parser;
ParserTimestampOperatorExpression next_parser;
const char * getName() const override{ return "INTERVAL operator expression"; }
const char * getName() const override { return "INTERVAL operator expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserAdditiveExpression : public IParserBase
{
private:
@ -186,7 +204,7 @@ private:
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserIntervalOperatorExpression>()};
protected:
const char * getName() const override{ return "additive expression"; }
const char * getName() const override { return "additive expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{

View File

@ -1,3 +1,3 @@
SET max_parser_depth = 4000;
SELECT x FROM (SELECT number % 16 = 0 ? nan : (number % 24 = 0 ? NULL : (number % 37 = 0 ? nan : (number % 34 = 0 ? nan : (number % 3 = 0 ? NULL : (number % 68 = 0 ? 42 : (number % 28 = 0 ? nan : (number % 46 = 0 ? nan : (number % 13 = 0 ? nan : (number % 27 = 0 ? NULL : (number % 39 = 0 ? NULL : (number % 27 = 0 ? NULL : (number % 30 = 0 ? NULL : (number % 72 = 0 ? NULL : (number % 36 = 0 ? NULL : (number % 51 = 0 ? NULL : (number % 58 = 0 ? nan : (number % 26 = 0 ? 42 : (number % 13 = 0 ? nan : (number % 12 = 0 ? NULL : (number % 22 = 0 ? nan : (number % 36 = 0 ? NULL : (number % 63 = 0 ? NULL : (number % 27 = 0 ? NULL : (number % 18 = 0 ? NULL : (number % 69 = 0 ? NULL : (number % 76 = 0 ? nan : (number % 42 = 0 ? NULL : (number % 9 = 0 ? NULL : (toFloat64(number)))))))))))))))))))))))))))))) AS x FROM system.numbers LIMIT 1001) ORDER BY x ASC NULLS FIRST;
SELECT x FROM (SELECT number % 22 = 0 ? nan : (number % 56 = 0 ? 42 : (number % 45 = 0 ? NULL : (number % 47 = 0 ? 42 : (number % 39 = 0 ? NULL : (number % 1 = 0 ? nan : (number % 43 = 0 ? nan : (number % 40 = 0 ? nan : (number % 42 = 0 ? NULL : (number % 26 = 0 ? 42 : (number % 41 = 0 ? 42 : (number % 6 = 0 ? NULL : (number % 39 = 0 ? NULL : (number % 34 = 0 ? nan : (number % 74 = 0 ? 42 : (number % 40 = 0 ? nan : (number % 37 = 0 ? nan : (number % 51 = 0 ? NULL : (number % 46 = 0 ? nan : (toFloat64(number)))))))))))))))))))) AS x FROM system.numbers LIMIT 1001) ORDER BY x ASC NULLS FIRST;
SELECT x FROM (SELECT number % 22 = 0 ? nan : (number % 56 = 0 ? 42 : (number % 45 = 0 ? NULL : (number % 47 = 0 ? 42 : (number % 39 = 0 ? NULL : (number % 1 = 0 ? nan : (number % 43 = 0 ? nan : (number % 40 = 0 ? nan : (number % 42 = 0 ? NULL : (number % 26 = 0 ? 42 : (number % 41 = 0 ? 42 : (number % 6 = 0 ? NULL : (number % 39 = 0 ? NULL : (number % 34 = 0 ? nan : (number % 74 = 0 ? 42 : (number % 40 = 0 ? nan : (number % 37 = 0 ? nan : (number % 51 = 0 ? NULL : (number % 46 = 0 ? nan : (toFloat64(number)))))))))))))))))))) AS x FROM system.numbers LIMIT 1001) ORDER BY x ASC NULLS FIRST;

View File

@ -1,4 +1,4 @@
-
Maximum parse depth (40) exceeded.
Maximum parse depth (42) exceeded.
-
Maximum parse depth (20) exceeded.

View File

@ -3,8 +3,8 @@
CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
. $CURDIR/../shell_config.sh
echo 'select 1' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&max_parser_depth=40" -d @- 2>&1 | grep -oP "Maximum parse depth .* exceeded."
echo 'select 1' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&max_parser_depth=42" -d @- 2>&1 | grep -oP "Maximum parse depth .* exceeded."
echo -
echo 'select (1+1)*(2+1)' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&max_parser_depth=40" -d @- 2>&1 | grep -oP "Maximum parse depth .* exceeded."
echo 'select (1+1)*(2+1)' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&max_parser_depth=42" -d @- 2>&1 | grep -oP "Maximum parse depth .* exceeded."
echo -
echo 'select 1' | ${CLICKHOUSE_CURL} -sSg "${CLICKHOUSE_URL}&max_parser_depth=20" -d @- 2>&1 | grep -oP "Maximum parse depth .* exceeded."

View File

@ -129,7 +129,7 @@ select
from
lineitem
where
l_shipdate <= toDate('1998-12-01') - interval 90 day
l_shipdate <= date '1998-12-01' - interval 90 day
group by
l_returnflag,
l_linestatus
@ -197,8 +197,8 @@ where
c_mktsegment = 'BUILDING'
and c_custkey = o_custkey
and l_orderkey = o_orderkey
and o_orderdate < toDate('1995-03-15')
and l_shipdate > toDate('1995-03-15')
and o_orderdate < date '1995-03-15'
and l_shipdate > date '1995-03-15'
group by
l_orderkey,
o_orderdate,
@ -215,8 +215,8 @@ limit 10;
-- from
-- orders
-- where
-- o_orderdate >= toDate('1993-07-01')
-- and o_orderdate < toDate('1993-07-01') + interval '3' month
-- o_orderdate >= date '1993-07-01'
-- and o_orderdate < date '1993-07-01' + interval '3' month
-- and exists (
-- select
-- *
@ -250,8 +250,8 @@ where
and s_nationkey = n_nationkey
and n_regionkey = r_regionkey
and r_name = 'ASIA'
and o_orderdate >= toDate('1994-01-01')
and o_orderdate < toDate('1994-01-01') + interval '1' year
and o_orderdate >= date '1994-01-01'
and o_orderdate < date '1994-01-01' + interval '1' year
group by
n_name
order by
@ -263,8 +263,8 @@ select
from
lineitem
where
l_shipdate >= toDate('1994-01-01')
and l_shipdate < toDate('1994-01-01') + interval '1' year
l_shipdate >= date '1994-01-01'
and l_shipdate < date '1994-01-01' + interval '1' year
and l_discount between toDecimal32(0.06, 2) - toDecimal32(0.01, 2)
and toDecimal32(0.06, 2) + toDecimal32(0.01, 2)
and l_quantity < 24;
@ -299,7 +299,7 @@ where
-- (n1.n_name = 'FRANCE' and n2.n_name = 'GERMANY')
-- or (n1.n_name = 'GERMANY' and n2.n_name = 'FRANCE')
-- )
-- and l_shipdate between toDate('1995-01-01') and toDate('1996-12-31')
-- and l_shipdate between date '1995-01-01' and date '1996-12-31'
-- ) as shipping
-- group by
-- supp_nation,
@ -341,7 +341,7 @@ where
-- and n1.n_regionkey = r_regionkey
-- and r_name = 'AMERICA'
-- and s_nationkey = n2.n_nationkey
-- and o_orderdate between toDate('1995-01-01') and toDate('1996-12-31')
-- and o_orderdate between date '1995-01-01' and date '1996-12-31'
-- and p_type = 'ECONOMY ANODIZED STEEL'
-- ) as all_nations
-- group by
@ -401,8 +401,8 @@ from
where
c_custkey = o_custkey
and l_orderkey = o_orderkey
and o_orderdate >= toDate('1993-10-01')
and o_orderdate < toDate('1993-10-01') + interval '3' month
and o_orderdate >= date '1993-10-01'
and o_orderdate < date '1993-10-01' + interval '3' month
and l_returnflag = 'R'
and c_nationkey = n_nationkey
group by
@ -472,8 +472,8 @@ where
and l_shipmode in ('MAIL', 'SHIP')
and l_commitdate < l_receiptdate
and l_shipdate < l_commitdate
and l_receiptdate >= toDate('1994-01-01')
and l_receiptdate < toDate('1994-01-01') + interval '1' year
and l_receiptdate >= date '1994-01-01'
and l_receiptdate < date '1994-01-01' + interval '1' year
group by
l_shipmode
order by
@ -513,8 +513,8 @@ from
part
where
l_partkey = p_partkey
and l_shipdate >= toDate('1995-09-01')
and l_shipdate < toDate('1995-09-01') + interval '1' month;
and l_shipdate >= date '1995-09-01'
and l_shipdate < date '1995-09-01' + interval '1' month;
-- select 15;
-- create view revenue0 as
@ -524,8 +524,8 @@ where
-- from
-- lineitem
-- where
-- l_shipdate >= toDate('1996-01-01')
-- and l_shipdate < toDate('1996-01-01') + interval '3' month
-- l_shipdate >= date '1996-01-01'
-- and l_shipdate < date '1996-01-01' + interval '3' month
-- group by
-- l_suppkey;
-- select
@ -702,8 +702,8 @@ where
-- where
-- l_partkey = ps_partkey
-- and l_suppkey = ps_suppkey
-- and l_shipdate >= toDate('1994-01-01')
-- and l_shipdate < toDate('1994-01-01') + interval '1' year
-- and l_shipdate >= date '1994-01-01'
-- and l_shipdate < date '1994-01-01' + interval '1' year
-- )
-- )
-- and s_nationkey = n_nationkey

View File

@ -0,0 +1,14 @@
1 1 1
1 1 1
1 1
2001-09-28
2001-10-05 Date
2001-09-24 Date
2001-10-05 Date
2001-09-24 Date
2001-09-28 01:00:00 DateTime
2001-09-27 23:00:00 DateTime
3 Int32
2001-09-29 00:00:00
2001-09-28 00:00:00
140400 Int32

View File

@ -0,0 +1,31 @@
select interval 1 second, interval 1 minute, interval 1 hour;
select interval 1 day, interval 1 week, interval 1 month;
select interval 1 quarter, interval 1 year;
select date '2001-09-28';
select (date '2001-09-28' + interval 7 day) x, toTypeName(x);
select (date '2001-10-01' - interval 7 day) x, toTypeName(x);
select (date '2001-09-28' + 7) x, toTypeName(x);
select (date '2001-10-01' - 7) x, toTypeName(x);
select (date '2001-09-28' + interval 1 hour) x, toTypeName(x);
select (date '2001-09-28' - interval 1 hour) x, toTypeName(x);
select (date '2001-10-01' - date '2001-09-28') x, toTypeName(x);
select timestamp '2001-09-28 01:00:00' + interval 23 hour;
select timestamp '2001-09-28 23:00:00' - interval 23 hour;
-- TODO: return interval
select (timestamp '2001-09-29 03:00:00' - timestamp '2001-09-27 12:00:00') x, toTypeName(x); -- interval '1 day 15:00:00'
-- select -interval 23 hour; -- interval '-23:00:00'
-- select interval 1 day + interval 1 hour; -- interval '1 day 01:00:00'
-- select interval '1 day' - interval '1 hour'; -- interval '1 day -01:00:00'
-- select date '2001-09-28' + time '03:00'; -- timestamp '2001-09-28 03:00:00'
-- select time '01:00' + interval '3 hours'; -- time '04:00:00'
-- select time '05:00' - time '03:00'; -- interval '02:00:00'
-- select time '05:00' - interval '2 hours'; -- time '03:00:00'
-- select 900 * interval '1 second'; -- interval '00:15:00'
-- select (21 * interval '1 day') x, toTypeName(x); -- interval '21 days'
-- select (double precision '3.5' * interval '1 hour') x, toTypeName(x); -- interval '03:30:00'
-- select (interval '1 hour' / double precision '1.5') x, toTypeName(x); -- interval '00:40:00'