Fixes & remove more of old parser

This commit is contained in:
Nikolay Degterinsky 2022-09-09 18:40:39 +00:00
parent f83aba3586
commit 4fb9bafb2e
5 changed files with 26 additions and 104 deletions

View File

@ -40,6 +40,7 @@
#include <Parsers/queryToString.h> #include <Parsers/queryToString.h>
#include <Interpreters/StorageID.h> #include <Interpreters/StorageID.h>
#include <Common/logger_useful.h>
namespace DB namespace DB
@ -53,89 +54,6 @@ namespace ErrorCodes
} }
bool ParserArray::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ASTPtr contents_node;
ParserExpressionList contents(false);
if (pos->type != TokenType::OpeningSquareBracket)
return false;
++pos;
if (!contents.parse(pos, contents_node, expected))
return false;
if (pos->type != TokenType::ClosingSquareBracket)
return false;
++pos;
auto function_node = std::make_shared<ASTFunction>();
function_node->name = "array";
function_node->arguments = contents_node;
function_node->children.push_back(contents_node);
node = function_node;
return true;
}
bool ParserParenthesisExpression::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
ASTPtr contents_node;
ParserExpressionList contents(false);
if (pos->type != TokenType::OpeningRoundBracket)
return false;
++pos;
if (!contents.parse(pos, contents_node, expected))
return false;
bool is_elem = true;
if (pos->type == TokenType::Comma)
{
is_elem = false;
++pos;
}
if (pos->type != TokenType::ClosingRoundBracket)
return false;
++pos;
const auto & expr_list = contents_node->as<ASTExpressionList &>();
/// Empty expression in parentheses is not allowed.
if (expr_list.children.empty())
{
expected.add(pos, "non-empty parenthesized list of expressions");
return false;
}
/// Special case for one-element tuple.
if (expr_list.children.size() == 1 && is_elem)
{
auto * ast_literal = expr_list.children.front()->as<ASTLiteral>();
/// But only if its argument is not tuple,
/// since otherwise it will do incorrect transformation:
///
/// (foo,bar) IN (('foo','bar')) -> (foo,bar) IN ('foo','bar')
if (!(ast_literal && ast_literal->value.getType() == Field::Types::Tuple))
{
node = expr_list.children.front();
return true;
}
}
auto function_node = std::make_shared<ASTFunction>();
function_node->name = "tuple";
function_node->arguments = contents_node;
function_node->children.push_back(contents_node);
node = function_node;
return true;
}
bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) bool ParserSubquery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{ {
ASTPtr select_node; ASTPtr select_node;
@ -1885,6 +1803,7 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
std::vector<CollectionOfLiteralsLayer<Collection>> layers; std::vector<CollectionOfLiteralsLayer<Collection>> layers;
layers.emplace_back(pos); layers.emplace_back(pos);
pos.increaseDepth();
ParserLiteral literal_p; ParserLiteral literal_p;
@ -1905,6 +1824,7 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
literal->end = ++pos; literal->end = ++pos;
layers.pop_back(); layers.pop_back();
pos.decreaseDepth();
if (layers.empty()) if (layers.empty())
{ {
@ -1913,6 +1833,7 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
} }
layers.back().arr.push_back(literal->value); layers.back().arr.push_back(literal->value);
continue;
} }
else if (pos->type == TokenType::Comma) else if (pos->type == TokenType::Comma)
{ {
@ -1931,9 +1852,14 @@ bool ParserCollectionOfLiterals<Collection>::parseImpl(Pos & pos, ASTPtr & node,
ASTPtr literal_node; ASTPtr literal_node;
if (literal_p.parse(pos, literal_node, expected)) if (literal_p.parse(pos, literal_node, expected))
{
layers.back().arr.push_back(literal_node->as<ASTLiteral &>().value); layers.back().arr.push_back(literal_node->as<ASTLiteral &>().value);
}
else if (pos->type == opening_bracket) else if (pos->type == opening_bracket)
{
layers.emplace_back(pos); layers.emplace_back(pos);
pos.increaseDepth();
}
else else
return false; return false;
} }

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. /** The SELECT subquery is in parenthesis.
*/ */
class ParserSubquery : public IParserBase class ParserSubquery : public IParserBase

View File

@ -2158,6 +2158,12 @@ bool ParserIntervalOperatorExpression::parseImpl(Pos & pos, ASTPtr & node, Expec
&& ParserExpressionImpl<IntervalLayer>().parse(pos, node, expected); && ParserExpressionImpl<IntervalLayer>().parse(pos, node, expected);
} }
bool ParserArray::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
{
return ParserToken(TokenType::OpeningSquareBracket).ignore(pos, expected)
&& ParserExpressionImpl<ArrayLayer>().parse(pos, node, expected);
}
template<class Type> template<class Type>
std::vector<std::pair<const char *, Operator>> ParserExpressionImpl<Type>::operators_table({ std::vector<std::pair<const char *, Operator>> ParserExpressionImpl<Type>::operators_table({
{"->", Operator("lambda", 1, 2, OperatorType::Lambda)}, {"->", Operator("lambda", 1, 2, OperatorType::Lambda)},
@ -2362,7 +2368,7 @@ typename ParserExpressionImpl<Type>::ParseResult ParserExpressionImpl<Type>::try
if (ParseDateOperatorExpression(pos, tmp, expected) || if (ParseDateOperatorExpression(pos, tmp, expected) ||
ParseTimestampOperatorExpression(pos, tmp, expected) || ParseTimestampOperatorExpression(pos, tmp, expected) ||
tuple_literal_parser.parse(pos, tmp, expected) || tuple_literal_parser.parse(pos, tmp, expected) ||
(layers.size() == 1 && array_literal_parser.parse(pos, tmp, expected)) || array_literal_parser.parse(pos, tmp, expected) ||
number_parser.parse(pos, tmp, expected) || number_parser.parse(pos, tmp, expected) ||
literal_parser.parse(pos, tmp, expected) || literal_parser.parse(pos, tmp, expected) ||
asterisk_parser.parse(pos, tmp, expected) || asterisk_parser.parse(pos, tmp, expected) ||

View File

@ -240,6 +240,15 @@ protected:
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override; bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
}; };
/// TODO: ?
class ParserArray : public IParserBase
{
protected:
const char * getName() const override { return "array"; }
bool parseImpl(Pos & pos, ASTPtr & node, Expected & expected) override;
};
// It's used to parse expressions in table function. // It's used to parse expressions in table function.
class ParserTableFunctionExpression : public IParserBase class ParserTableFunctionExpression : public IParserBase
{ {

View File

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