mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
Add DATE and TIMESTAMP operators (#9691)
* DATE and TIMESTAMP operators
This commit is contained in:
parent
c19c954f1e
commit
dbc08d7644
@ -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)
|
||||
{
|
||||
|
@ -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
|
||||
{
|
||||
|
@ -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;
|
||||
|
@ -1,4 +1,4 @@
|
||||
-
|
||||
Maximum parse depth (40) exceeded.
|
||||
Maximum parse depth (42) exceeded.
|
||||
-
|
||||
Maximum parse depth (20) exceeded.
|
||||
|
@ -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."
|
||||
|
@ -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
|
||||
|
@ -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
|
@ -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'
|
Loading…
Reference in New Issue
Block a user