mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-10 01:25:21 +00:00
Merge pull request #34892 from evillique/new_parser
Implement operator precedence parser
This commit is contained in:
commit
a3b28b441b
File diff suppressed because it is too large
Load Diff
@ -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
@ -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;
|
||||
|
||||
|
@ -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;
|
||||
|
@ -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.
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
|
@ -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>
|
||||
|
@ -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
|
||||
|
@ -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 + ")",
|
||||
|
@ -1,6 +1,4 @@
|
||||
exceeded
|
||||
exceeded
|
||||
exceeded
|
||||
exceeded
|
||||
20002
|
||||
1
|
||||
|
@ -1,4 +1,5 @@
|
||||
#!/usr/bin/env bash
|
||||
# Tags: no-asan
|
||||
|
||||
# Such a huge timeout mostly for debug build.
|
||||
CLICKHOUSE_CURL_TIMEOUT=60
|
||||
|
@ -1,4 +1,3 @@
|
||||
-
|
||||
Maximum parse depth (42) exceeded.
|
||||
-
|
||||
Maximum parse depth (20) exceeded.
|
||||
-
|
||||
|
@ -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 -
|
||||
|
@ -1,4 +1,5 @@
|
||||
Code: 306
|
||||
Code: 306
|
||||
Code: 306
|
||||
Code: 306
|
||||
Code: 167
|
||||
Code: 167
|
||||
Code: 167
|
||||
Code: 167
|
||||
Code: 167
|
||||
|
@ -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'
|
||||
|
Loading…
Reference in New Issue
Block a user