Merge pull request #34892 from evillique/new_parser

Implement operator precedence parser
This commit is contained in:
Alexey Milovidov 2022-09-27 00:07:04 +03:00 committed by GitHub
commit a3b28b441b
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
19 changed files with 2229 additions and 2006 deletions

File diff suppressed because it is too large Load Diff

View File

@ -9,26 +9,6 @@ namespace DB
{
class ParserArray : public IParserBase
{
protected:
const char * getName() const override { return "array"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** If in parenthesis an expression from one element - returns this element in `node`;
* or if there is a SELECT subquery in parenthesis, then this subquery returned in `node`;
* otherwise returns `tuple` function from the contents of brackets.
*/
class ParserParenthesisExpression : public IParserBase
{
protected:
const char * getName() const override { return "parenthesized expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** The SELECT subquery is in parenthesis.
*/
class ParserSubquery : public IParserBase
@ -141,36 +121,6 @@ protected:
ColumnTransformers allowed_transformers;
};
/** A function, for example, f(x, y + 1, g(z)).
* Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function.
* Or a parametric aggregate function: quantile(0.9)(x + y).
* Syntax - two pairs of parentheses instead of one. The first is for parameters, the second for arguments.
* For functions, the DISTINCT modifier can be specified, for example, count(DISTINCT x, y).
*/
class ParserFunction : public IParserBase
{
public:
explicit ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false)
: allow_function_parameters(allow_function_parameters_), is_table_function(is_table_function_)
{
}
protected:
const char * getName() const override { return "function"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
bool allow_function_parameters;
bool is_table_function;
};
// A special function parser for view and viewIfPermitted table functions.
// It parses an SELECT query as its argument and doesn't support getColumnName().
class ParserTableFunctionView : public IParserBase
{
protected:
const char * getName() const override { return "function"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
// Allows to make queries like SELECT SUM(<expr>) FILTER(WHERE <cond>) FROM ...
class ParserFilterClause : public IParserBase
{
@ -394,16 +344,6 @@ protected:
};
/** The expression element is one of: an expression in parentheses, an array, a literal, a function, an identifier, an asterisk.
*/
class ParserExpressionElement : public IParserBase
{
protected:
const char * getName() const override { return "element of expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** An expression element, possibly with an alias, if appropriate.
*/
class ParserWithOptionalAlias : public IParserBase

File diff suppressed because it is too large Load Diff

View File

@ -116,6 +116,36 @@ private:
SelectUnionModes union_modes;
};
class ParserArray : public IParserBase
{
protected:
const char * getName() const override { return "array"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** A function, for example, f(x, y + 1, g(z)).
* Or an aggregate function: sum(x + f(y)), corr(x, y). The syntax is the same as the usual function.
* Or a parametric aggregate function: quantile(0.9)(x + y).
* Syntax - two pairs of parentheses instead of one. The first is for parameters, the second for arguments.
* For functions, the DISTINCT modifier can be specified, for example, count(DISTINCT x, y).
*/
class ParserFunction : public IParserBase
{
public:
explicit ParserFunction(bool allow_function_parameters_ = true, bool is_table_function_ = false)
: allow_function_parameters(allow_function_parameters_), is_table_function(is_table_function_)
{
}
protected:
const char * getName() const override { return "function"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
bool allow_function_parameters;
bool is_table_function;
};
/** An expression with an infix binary left-associative operator.
* For example, a + b - c + d.
*/
@ -123,31 +153,13 @@ class ParserLeftAssociativeBinaryOperatorList : public IParserBase
{
private:
Operators_t operators;
Operators_t overlapping_operators_to_skip = { (const char *[]){ nullptr } };
ParserPtr first_elem_parser;
ParserPtr remaining_elem_parser;
/// =, !=, <, > ALL (subquery) / ANY (subquery)
bool comparison_expression = false;
ParserPtr elem_parser;
public:
/** `operators_` - allowed operators and their corresponding functions
*/
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && first_elem_parser_)
: operators(operators_), first_elem_parser(std::move(first_elem_parser_))
{
}
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_,
Operators_t overlapping_operators_to_skip_, ParserPtr && first_elem_parser_, bool comparison_expression_ = false)
: operators(operators_), overlapping_operators_to_skip(overlapping_operators_to_skip_),
first_elem_parser(std::move(first_elem_parser_)), comparison_expression(comparison_expression_)
{
}
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && first_elem_parser_,
ParserPtr && remaining_elem_parser_)
: operators(operators_), first_elem_parser(std::move(first_elem_parser_)),
remaining_elem_parser(std::move(remaining_elem_parser_))
ParserLeftAssociativeBinaryOperatorList(Operators_t operators_, ParserPtr && elem_parser_)
: operators(operators_), elem_parser(std::move(elem_parser_))
{
}
@ -158,295 +170,8 @@ protected:
};
/** Expression with an infix operator of arbitrary arity.
* For example, a AND b AND c AND d.
*/
class ParserVariableArityOperatorList : public IParserBase
class ParserExpression : public IParserBase
{
private:
const char * infix;
const char * function_name;
ParserPtr elem_parser;
public:
ParserVariableArityOperatorList(const char * infix_, const char * function_, ParserPtr && elem_parser_)
: infix(infix_), function_name(function_), elem_parser(std::move(elem_parser_))
{
}
protected:
const char * getName() const override { return "list, delimited by operator of variable arity"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/** An expression with a prefix unary operator.
* Example, NOT x.
*/
class ParserPrefixUnaryOperatorExpression : public IParserBase
{
private:
Operators_t operators;
ParserPtr elem_parser;
public:
/** `operators_` - allowed operators and their corresponding functions
*/
ParserPrefixUnaryOperatorExpression(Operators_t operators_, ParserPtr && elem_parser_)
: operators(operators_), elem_parser(std::move(elem_parser_))
{
}
protected:
const char * getName() const override { return "expression with prefix unary operator"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
/// CAST operator "::". This parser is used if left argument
/// of operator cannot be read as simple literal from text of query.
/// Example: "[1, 1 + 1, 1 + 2]::Array(UInt8)"
class ParserCastExpression : public IParserBase
{
private:
ParserPtr elem_parser;
public:
explicit ParserCastExpression(ParserPtr && elem_parser_)
: elem_parser(std::move(elem_parser_))
{
}
protected:
const char * getName() const override { return "CAST expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserArrayElementExpression : public IParserBase
{
private:
static const char * operators[];
protected:
const char * getName() const override{ return "array element expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserTupleElementExpression : public IParserBase
{
private:
static const char * operators[];
protected:
const char * getName() const override { return "tuple element expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserUnaryExpression : public IParserBase
{
private:
static const char * operators[];
ParserPrefixUnaryOperatorExpression operator_parser {operators, std::make_unique<ParserCastExpression>(std::make_unique<ParserTupleElementExpression>())};
protected:
const char * getName() const override { return "unary expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserMultiplicativeExpression : public IParserBase
{
private:
static const char * operators[];
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserUnaryExpression>()};
protected:
const char * getName() const override { return "multiplicative expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
/// 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:
ParserTimestampOperatorExpression next_parser;
const char * getName() const override { return "INTERVAL operator expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
private:
static bool parseArgumentAndIntervalKind(Pos & pos, ASTPtr & expr, IntervalKind & interval_kind, Expected & expected);
};
class ParserAdditiveExpression : public IParserBase
{
private:
static const char * operators[];
ParserLeftAssociativeBinaryOperatorList operator_parser {operators, std::make_unique<ParserIntervalOperatorExpression>()};
protected:
const char * getName() const override { return "additive expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
class ParserConcatExpression : public IParserBase
{
ParserVariableArityOperatorList operator_parser {"||", "concat", std::make_unique<ParserAdditiveExpression>()};
protected:
const char * getName() const override { return "string concatenation expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
class ParserBetweenExpression : public IParserBase
{
private:
ParserConcatExpression elem_parser;
protected:
const char * getName() const override { return "BETWEEN expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserComparisonExpression : public IParserBase
{
private:
static const char * operators[];
static const char * overlapping_operators_to_skip[];
ParserLeftAssociativeBinaryOperatorList operator_parser {operators,
overlapping_operators_to_skip, std::make_unique<ParserBetweenExpression>(), true};
protected:
const char * getName() const override{ return "comparison expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
/** Parser for nullity checking with IS (NOT) NULL.
*/
class ParserNullityChecking : public IParserBase
{
private:
ParserComparisonExpression elem_parser;
protected:
const char * getName() const override { return "nullity checking"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserLogicalNotExpression : public IParserBase
{
private:
static const char * operators[];
ParserPrefixUnaryOperatorExpression operator_parser {operators, std::make_unique<ParserNullityChecking>()};
protected:
const char * getName() const override{ return "logical-NOT expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
class ParserLogicalAndExpression : public IParserBase
{
private:
ParserVariableArityOperatorList operator_parser {"AND", "and", std::make_unique<ParserLogicalNotExpression>()};
protected:
const char * getName() const override { return "logical-AND expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
class ParserLogicalOrExpression : public IParserBase
{
private:
ParserVariableArityOperatorList operator_parser {"OR", "or", std::make_unique<ParserLogicalAndExpression>()};
protected:
const char * getName() const override { return "logical-OR expression"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override
{
return operator_parser.parse(pos, node, expected);
}
};
/** An expression with ternary operator.
* For example, a = 1 ? b + 1 : c * 2.
*/
class ParserTernaryOperatorExpression : public IParserBase
{
private:
ParserLogicalOrExpression elem_parser;
protected:
const char * getName() const override { return "expression with ternary operator"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
class ParserLambdaExpression : public IParserBase
{
private:
ParserTernaryOperatorExpression elem_parser;
protected:
const char * getName() const override { return "lambda expression"; }
@ -457,9 +182,6 @@ protected:
// It's used to parse expressions in table function.
class ParserTableFunctionExpression : public IParserBase
{
private:
ParserLambdaExpression elem_parser;
protected:
const char * getName() const override { return "table function expression"; }
@ -467,13 +189,10 @@ protected:
};
using ParserExpression = ParserLambdaExpression;
class ParserExpressionWithOptionalAlias : public IParserBase
{
public:
explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword, bool is_table_function = false);
explicit ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword_, bool is_table_function_ = false);
protected:
ParserPtr impl;

View File

@ -138,14 +138,28 @@ void IAST::updateTreeHashImpl(SipHash & hash_state) const
}
size_t IAST::checkDepthImpl(size_t max_depth, size_t level) const
size_t IAST::checkDepthImpl(size_t max_depth) const
{
size_t res = level + 1;
for (const auto & child : children)
std::vector<std::pair<ASTPtr, size_t>> stack;
stack.reserve(children.size());
for (const auto & i: children)
stack.push_back({i, 1});
size_t res = 0;
while (!stack.empty())
{
if (level >= max_depth)
auto top = stack.back();
stack.pop_back();
if (top.second >= max_depth)
throw Exception("AST is too deep. Maximum: " + toString(max_depth), ErrorCodes::TOO_DEEP_AST);
res = std::max(res, child->checkDepthImpl(max_depth, level + 1));
res = std::max(res, top.second);
for (const auto & i: top.first->children)
stack.push_back({i, top.second + 1});
}
return res;

View File

@ -92,7 +92,7 @@ public:
*/
size_t checkDepth(size_t max_depth) const
{
return checkDepthImpl(max_depth, 0);
return checkDepthImpl(max_depth);
}
/** Get total number of tree elements
@ -273,7 +273,7 @@ public:
static const char * hilite_none;
private:
size_t checkDepthImpl(size_t max_depth, size_t level) const;
size_t checkDepthImpl(size_t max_depth) const;
/** Forward linked list of ASTPtr to delete.
* Used in IAST destructor to avoid possible stack overflow.

View File

@ -20,7 +20,7 @@ bool ParserCreateFunctionQuery::parseImpl(IParser::Pos & pos, ASTPtr & node, Exp
ParserKeyword s_on("ON");
ParserIdentifier function_name_p;
ParserKeyword s_as("AS");
ParserLambdaExpression lambda_p;
ParserExpression lambda_p;
ASTPtr function_name;
ASTPtr function_core;

View File

@ -152,7 +152,7 @@ bool ParserConstraintDeclaration::parseImpl(Pos & pos, ASTPtr & node, Expected &
ParserKeyword s_assume("ASSUME");
ParserIdentifier name_p;
ParserLogicalOrExpression expression_p;
ParserExpression expression_p;
ASTPtr name;
ASTPtr expr;
@ -858,8 +858,8 @@ bool ParserCreateWindowViewQuery::parseImpl(Pos & pos, ASTPtr & node, Expected &
ParserStorage storage_p;
ParserStorage storage_inner;
ParserTablePropertiesDeclarationList table_properties_p;
ParserIntervalOperatorExpression watermark_p;
ParserIntervalOperatorExpression lateness_p;
ParserExpression watermark_p;
ParserExpression lateness_p;
ParserSelectWithUnionQuery select_p;
ASTPtr table;

View File

@ -134,7 +134,7 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, ASTPtr & node, E
ParserKeyword s_remove{"REMOVE"};
ParserKeyword s_type{"TYPE"};
ParserKeyword s_collate{"COLLATE"};
ParserTernaryOperatorExpression expr_parser;
ParserExpression expr_parser;
ParserStringLiteral string_literal_parser;
ParserLiteral literal_parser;
ParserCodec codec_parser;

View File

@ -20,7 +20,7 @@ bool ParserDictionaryAttributeDeclaration::parseImpl(Pos & pos, ASTPtr & node, E
ParserKeyword s_is_object_id{"IS_OBJECT_ID"};
ParserLiteral default_parser;
ParserArrayOfLiterals array_literals_parser;
ParserTernaryOperatorExpression expression_parser;
ParserExpression expression_parser;
/// mandatory attribute name
ASTPtr name;

View File

@ -3,7 +3,7 @@
#include <Parsers/ASTExternalDDLQuery.h>
#include <Parsers/ASTSelectWithUnionQuery.h>
#include <Parsers/CommonParsers.h>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ExpressionListParsers.h>
#include <Parsers/ParserDropQuery.h>
#include <Parsers/ParserExternalDDLQuery.h>
#include <Parsers/ParserRenameQuery.h>

View File

@ -226,7 +226,7 @@ bool ParserTablesInSelectQueryElement::parseImpl(Pos & pos, ASTPtr & node, Expec
else if (ParserKeyword("ON").ignore(pos, expected))
{
/// OR is operator with lowest priority, so start parsing from it.
if (!ParserLogicalOrExpression().parse(pos, table_join->on_expression, expected))
if (!ParserExpression().parse(pos, table_join->on_expression, expected))
return false;
}
else

View File

@ -61,7 +61,7 @@ namespace DB
ColumnsDescription /*cached_columns_*/) const
{
const Settings & settings = context_->getSettings();
ParserLambdaExpression partition_by_parser;
ParserExpression partition_by_parser;
ASTPtr partition_by_ast = parseQuery(
partition_by_parser,
"(" + partition_by_def + ")",

View File

@ -1,6 +1,4 @@
exceeded
exceeded
exceeded
exceeded
20002
1

View File

@ -1,4 +1,5 @@
#!/usr/bin/env bash
# Tags: no-asan
# Such a huge timeout mostly for debug build.
CLICKHOUSE_CURL_TIMEOUT=60

View File

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

View File

@ -9,3 +9,4 @@ echo -
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."
echo -

View File

@ -1,4 +1,5 @@
Code: 306
Code: 306
Code: 306
Code: 306
Code: 167
Code: 167
Code: 167
Code: 167
Code: 167

View File

@ -6,6 +6,6 @@ CURDIR=$(cd "$(dirname "${BASH_SOURCE[0]}")" && pwd)
{ printf "select "; for _ in {1..1000}; do printf "coalesce(null, "; done; printf "1"; for _ in {1..1000}; do printf ")"; done; } > "${CLICKHOUSE_TMP}"/query
cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_CLIENT 2>&1 | grep -o -F 'Code: 306'
cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_LOCAL 2>&1 | grep -o -F 'Code: 306'
cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_CURL --data-binary @- -vsS "$CLICKHOUSE_URL" 2>&1 | grep -o -F 'Code: 306'
cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_CLIENT 2>&1 | grep -o -F 'Code: 167'
cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_LOCAL 2>&1 | grep -o -F 'Code: 167'
cat "${CLICKHOUSE_TMP}"/query | $CLICKHOUSE_CURL --data-binary @- -vsS "$CLICKHOUSE_URL" 2>&1 | grep -o -F 'Code: 167'