提交 1bab7fc3 编写于 作者: A Alexey Milovidov 提交者: alexey-milovidov

Using lexer (incomplete) [#CLICKHOUSE-2].

上级 d815b766
...@@ -4,11 +4,7 @@ ...@@ -4,11 +4,7 @@
#include <DataTypes/DataTypeDateTime.h> #include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypesNumber.h> #include <DataTypes/DataTypesNumber.h>
#include <Columns/ColumnsNumber.h> #include <Columns/ColumnsNumber.h>
#include <Parsers/CommonParsers.h>
#include <ext/range.h> #include <ext/range.h>
#include <boost/range/iterator_range_core.hpp>
#include <Parsers/ExpressionElementParsers.h>
#include <Parsers/ASTLiteral.h>
#include <Common/PODArray.h> #include <Common/PODArray.h>
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
...@@ -164,23 +160,20 @@ public: ...@@ -164,23 +160,20 @@ public:
if (!sufficientArgs(arg_count)) if (!sufficientArgs(arg_count))
throw Exception{ throw Exception{
"Aggregate function " + getName() + " requires at least 3 arguments.", "Aggregate function " + getName() + " requires at least 3 arguments.",
ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION ErrorCodes::TOO_LESS_ARGUMENTS_FOR_FUNCTION};
};
if (arg_count - 1 > Data::max_events) if (arg_count - 1 > Data::max_events)
throw Exception{ throw Exception{
"Aggregate function " + getName() + " supports up to " + "Aggregate function " + getName() + " supports up to " +
std::to_string(Data::max_events) + " event arguments.", std::to_string(Data::max_events) + " event arguments.",
ErrorCodes::TOO_MUCH_ARGUMENTS_FOR_FUNCTION ErrorCodes::TOO_MUCH_ARGUMENTS_FOR_FUNCTION};
};
const auto time_arg = arguments.front().get(); const auto time_arg = arguments.front().get();
if (!typeid_cast<const DataTypeDateTime *>(time_arg)) if (!typeid_cast<const DataTypeDateTime *>(time_arg))
throw Exception{ throw Exception{
"Illegal type " + time_arg->getName() + " of first argument of aggregate function " + "Illegal type " + time_arg->getName() + " of first argument of aggregate function " +
getName() + ", must be DateTime", getName() + ", must be DateTime",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
};
for (const auto i : ext::range(1, arg_count)) for (const auto i : ext::range(1, arg_count))
{ {
...@@ -189,8 +182,7 @@ public: ...@@ -189,8 +182,7 @@ public:
throw Exception{ throw Exception{
"Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) + "Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) +
" of aggregate function " + getName() + ", must be UInt8", " of aggregate function " + getName() + ", must be UInt8",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
};
} }
parsePattern(); parsePattern();
...@@ -275,52 +267,51 @@ private: ...@@ -275,52 +267,51 @@ private:
actions.clear(); actions.clear();
actions.emplace_back(PatternActionType::KleeneStar); actions.emplace_back(PatternActionType::KleeneStar);
ParserString special_open_p("(?");
ParserString special_close_p(")");
ParserString t_p("t");
ParserString less_or_equal_p("<=");
ParserString less_p("<");
ParserString greater_or_equal_p(">=");
ParserString greater_p(">");
ParserString dot_closure_p(".*");
ParserString dot_p(".");
ParserNumber number_p;
const char * pos = pattern.data(); const char * pos = pattern.data();
const auto begin = pos; const char * begin = pos;
const auto end = pos + pattern.size(); const char * end = pos + pattern.size();
ASTPtr node;
decltype(pos) max_parsed_pos{};
Expected expected;
const auto throw_exception = [&] (const std::string & msg) auto throw_exception = [&](const std::string & msg)
{ {
throw Exception{ throw Exception{
msg + " '" + std::string(pos, end) + "' at position " + std::to_string(pos - begin), msg + " '" + std::string(pos, end) + "' at position " + std::to_string(pos - begin),
ErrorCodes::SYNTAX_ERROR}; ErrorCodes::SYNTAX_ERROR};
}; };
auto match = [&](const char * str)
{
size_t length = strlen(str);
if (pos + length < end && 0 == memcmp(pos, str, length))
{
pos += length;
return true;
}
return false;
};
while (pos < end) while (pos < end)
{ {
if (special_open_p.ignore(pos, end)) if (match("(?"))
{ {
if (t_p.ignore(pos, end)) if (match("t"))
{ {
PatternActionType type; PatternActionType type;
if (less_or_equal_p.ignore(pos, end)) if (match("<="))
type = PatternActionType::TimeLessOrEqual; type = PatternActionType::TimeLessOrEqual;
else if (less_p.ignore(pos, end)) else if (match("<"))
type = PatternActionType::TimeLess; type = PatternActionType::TimeLess;
else if (greater_or_equal_p.ignore(pos, end)) else if (match(">="))
type = PatternActionType::TimeGreaterOrEqual; type = PatternActionType::TimeGreaterOrEqual;
else if (greater_p.ignore(pos, end)) else if (match(">"))
type = PatternActionType::TimeGreater; type = PatternActionType::TimeGreater;
else else
throw_exception("Unknown time condition"); throw_exception("Unknown time condition");
if (!number_p.parse(pos, node, expected)) UInt64 duration = 0;
auto prev_pos = pos;
pos = tryReadIntText(duration, pos, end);
if (pos == prev_pos)
throw_exception("Could not parse number"); throw_exception("Could not parse number");
if (actions.back().type != PatternActionType::SpecificEvent && if (actions.back().type != PatternActionType::SpecificEvent &&
...@@ -328,32 +319,33 @@ private: ...@@ -328,32 +319,33 @@ private:
actions.back().type != PatternActionType::KleeneStar) actions.back().type != PatternActionType::KleeneStar)
throw Exception{ throw Exception{
"Temporal condition should be preceeded by an event condition", "Temporal condition should be preceeded by an event condition",
ErrorCodes::BAD_ARGUMENTS ErrorCodes::BAD_ARGUMENTS};
};
actions.emplace_back(type, typeid_cast<const ASTLiteral &>(*node).value.safeGet<UInt64>()); actions.emplace_back(type, duration);
} }
else if (number_p.parse(pos, node, expected)) else
{ {
const auto event_number = typeid_cast<const ASTLiteral &>(*node).value.safeGet<UInt64>(); UInt64 event_number = 0;
auto prev_pos = pos;
pos = tryReadIntText(event_number, pos, end);
if (pos == prev_pos)
throw_exception("Could not parse number");
if (event_number > arg_count - 1) if (event_number > arg_count - 1)
throw Exception{ throw Exception{
"Event number " + std::to_string(event_number) + " is out of range", "Event number " + std::to_string(event_number) + " is out of range",
ErrorCodes::BAD_ARGUMENTS ErrorCodes::BAD_ARGUMENTS};
};
actions.emplace_back(PatternActionType::SpecificEvent, event_number - 1); actions.emplace_back(PatternActionType::SpecificEvent, event_number - 1);
} }
else
throw_exception("Unexpected special sequence");
if (!special_close_p.ignore(pos, end)) if (!match(")"))
throw_exception("Expected closing parenthesis, found"); throw_exception("Expected closing parenthesis, found");
} }
else if (dot_closure_p.ignore(pos, end)) else if (match(".*"))
actions.emplace_back(PatternActionType::KleeneStar); actions.emplace_back(PatternActionType::KleeneStar);
else if (dot_p.ignore(pos, end)) else if (match("."))
actions.emplace_back(PatternActionType::AnyEvent); actions.emplace_back(PatternActionType::AnyEvent);
else else
throw_exception("Could not parse pattern, unexpected starting symbol"); throw_exception("Could not parse pattern, unexpected starting symbol");
......
#include <IO/ReadHelpers.h> #include <IO/ReadHelpers.h>
#include <Interpreters/evaluateConstantExpression.h> #include <Interpreters/evaluateConstantExpression.h>
#include <Interpreters/convertFieldToType.h> #include <Interpreters/convertFieldToType.h>
#include <Parsers/TokenIterator.h>
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionListParsers.h>
#include <DataStreams/ValuesRowInputStream.h> #include <DataStreams/ValuesRowInputStream.h>
#include <DataTypes/DataTypeArray.h> #include <DataTypes/DataTypeArray.h>
...@@ -85,7 +86,7 @@ bool ValuesRowInputStream::read(Block & block) ...@@ -85,7 +86,7 @@ bool ValuesRowInputStream::read(Block & block)
|| e.code() == ErrorCodes::CANNOT_PARSE_DATETIME || e.code() == ErrorCodes::CANNOT_PARSE_DATETIME
|| e.code() == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT) || e.code() == ErrorCodes::CANNOT_READ_ARRAY_FROM_TEXT)
{ {
/// TODO Performance if the expression does not fit entirely to the end of the buffer. /// TODO Case when the expression does not fit entirely in the buffer.
/// If the beginning of the value is no longer in the buffer. /// If the beginning of the value is no longer in the buffer.
if (istr.count() - istr.offset() != prev_istr_bytes) if (istr.count() - istr.offset() != prev_istr_bytes)
...@@ -96,18 +97,18 @@ bool ValuesRowInputStream::read(Block & block) ...@@ -96,18 +97,18 @@ bool ValuesRowInputStream::read(Block & block)
IDataType & type = *block.safeGetByPosition(i).type; IDataType & type = *block.safeGetByPosition(i).type;
IParser::Pos pos = prev_istr_position;
Expected expected = ""; Expected expected = "";
IParser::Pos max_parsed_pos = pos;
Tokens tokens(prev_istr_position, istr.buffer().end());
TokenIterator token_iterator(tokens);
ASTPtr ast; ASTPtr ast;
if (!parser.parse(pos, istr.buffer().end(), ast, max_parsed_pos, expected)) if (!parser.parse(token_iterator, ast, expected))
throw Exception("Cannot parse expression of type " + type.getName() + " here: " throw Exception("Cannot parse expression of type " + type.getName() + " here: "
+ String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)), + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)),
ErrorCodes::SYNTAX_ERROR); ErrorCodes::SYNTAX_ERROR);
istr.position() = const_cast<char *>(max_parsed_pos); istr.position() = const_cast<char *>(token_iterator->begin);
std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(ast, context); std::pair<Field, DataTypePtr> value_raw = evaluateConstantExpression(ast, context);
Field value = convertFieldToType(value_raw.first, type, value_raw.second.get()); Field value = convertFieldToType(value_raw.first, type, value_raw.second.get());
......
...@@ -71,7 +71,7 @@ DataTypeFactory::DataTypeFactory() ...@@ -71,7 +71,7 @@ DataTypeFactory::DataTypeFactory()
template <typename DataTypeEnum> template <typename DataTypeEnum>
inline DataTypePtr parseEnum(const String & name, const String & base_name, const String & parameters) inline DataTypePtr parseEnum(const String & name, const String & base_name, const String & parameters)
{ {
ParserList parser{std::make_unique<ParserEnumElement>(), std::make_unique<ParserString>(","), false}; ParserList parser{std::make_unique<ParserEnumElement>(), std::make_unique<ParserToken>(TokenType::Comma), false};
ASTPtr elements = parseQuery(parser, parameters.data(), parameters.data() + parameters.size(), "parameters for enum type " + name); ASTPtr elements = parseQuery(parser, parameters.data(), parameters.data() + parameters.size(), "parameters for enum type " + name);
......
...@@ -263,9 +263,9 @@ void DDLWorker::processTask(const DDLLogEntry & node, const std::string & node_n ...@@ -263,9 +263,9 @@ void DDLWorker::processTask(const DDLLogEntry & node, const std::string & node_n
{ {
ASTPtr query_ast; ASTPtr query_ast;
{ {
ParserQuery parser_query;
String description;
const char * begin = &node.query.front(); const char * begin = &node.query.front();
ParserQuery parser_query(begin + node.query.size());
String description;
query_ast = parseQuery(parser_query, begin, begin + node.query.size(), description); query_ast = parseQuery(parser_query, begin, begin + node.query.size(), description);
} }
......
...@@ -131,7 +131,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl( ...@@ -131,7 +131,7 @@ static std::tuple<ASTPtr, BlockIO> executeQueryImpl(
const Settings & settings = context.getSettingsRef(); const Settings & settings = context.getSettingsRef();
ParserQuery parser; ParserQuery parser(end);
ASTPtr ast; ASTPtr ast;
size_t query_size; size_t query_size;
size_t max_query_size = settings.max_query_size; size_t max_query_size = settings.max_query_size;
......
...@@ -533,13 +533,13 @@ bool ParserTupleElementExpression::parseImpl(Pos & pos, ASTPtr & node, Expected ...@@ -533,13 +533,13 @@ bool ParserTupleElementExpression::parseImpl(Pos & pos, ASTPtr & node, Expected
ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword) ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword)
: impl(std::make_unique<ParserWithOptionalAlias>(std::make_unique<ParserLambdaExpression>(), allow_alias_without_as_keyword)) : impl(std::make_unique<ParserWithOptionalAlias>(std::make_unique<ParserExpression>(), allow_alias_without_as_keyword))
{ {
} }
ParserExpressionInCastExpression::ParserExpressionInCastExpression(bool allow_alias_without_as_keyword) ParserExpressionInCastExpression::ParserExpressionInCastExpression(bool allow_alias_without_as_keyword)
: impl(std::make_unique<ParserCastExpressionWithOptionalAlias>(std::make_unique<ParserLambdaExpression>(), allow_alias_without_as_keyword)) : impl(std::make_unique<ParserCastExpressionWithOptionalAlias>(std::make_unique<ParserExpression>(), allow_alias_without_as_keyword))
{ {
} }
......
...@@ -305,6 +305,9 @@ protected: ...@@ -305,6 +305,9 @@ protected:
}; };
using ParserExpression = ParserLambdaExpression;
class ParserExpressionWithOptionalAlias : public IParserBase class ParserExpressionWithOptionalAlias : public IParserBase
{ {
public: public:
......
...@@ -217,7 +217,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ...@@ -217,7 +217,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
else if (s_reshard.ignore(pos, expected)) else if (s_reshard.ignore(pos, expected))
{ {
ParserList weighted_zookeeper_paths_p(std::make_unique<ParserWeightedZooKeeperPath>(), std::make_unique<ParserToken>(TokenType::Comma), false); ParserList weighted_zookeeper_paths_p(std::make_unique<ParserWeightedZooKeeperPath>(), std::make_unique<ParserToken>(TokenType::Comma), false);
ParserExpressionWithOptionalAlias parser_sharding_key_expr(false); ParserExpression parser_sharding_key_expr(false);
ParserStringLiteral parser_coordinator; ParserStringLiteral parser_coordinator;
if (s_copy.ignore(pos, expected)) if (s_copy.ignore(pos, expected))
......
...@@ -17,7 +17,7 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ...@@ -17,7 +17,7 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserKeyword s_then{"THEN"}; ParserKeyword s_then{"THEN"};
ParserKeyword s_else{"ELSE"}; ParserKeyword s_else{"ELSE"};
ParserKeyword s_end{ "END"}; ParserKeyword s_end{ "END"};
ParserExpressionWithOptionalAlias p_expr{false}; ParserExpression p_expr{false};
if (!s_case.parse(pos, node, expected)) if (!s_case.parse(pos, node, expected))
{ {
......
...@@ -27,7 +27,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect ...@@ -27,7 +27,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect
if (!ParserKeyword{"WHERE"}.ignore(pos, expected)) if (!ParserKeyword{"WHERE"}.ignore(pos, expected))
return false; return false;
ParserExpressionWithOptionalAlias p_where_expression(false); ParserExpression p_where_expression(false);
if (!p_where_expression.parse(pos, query->where_expression, expected)) if (!p_where_expression.parse(pos, query->where_expression, expected))
return false; return false;
......
...@@ -44,7 +44,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ...@@ -44,7 +44,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected)
ParserNotEmptyExpressionList exp_list(false); ParserNotEmptyExpressionList exp_list(false);
ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword. ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword.
ParserExpressionWithOptionalAlias exp_elem(false); ParserExpression exp_elem(false);
ParserOrderByExpressionList order_list; ParserOrderByExpressionList order_list;
/// SELECT [DISTINCT] expr list /// SELECT [DISTINCT] expr list
......
#include <Parsers/ExpressionListParsers.h> #include <Parsers/ExpressionListParsers.h>
#include <Parsers/parseQuery.h>
#include <Parsers/queryToString.h> #include <Parsers/queryToString.h>
#include <IO/WriteBuffer.h> #include <IO/WriteBuffer.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
...@@ -71,7 +72,7 @@ ColumnsDescription<true> ColumnsDescription<true>::parse(const String & str) ...@@ -71,7 +72,7 @@ ColumnsDescription<true> ColumnsDescription<true>::parse(const String & str)
readText(count, buf); readText(count, buf);
assertString(" columns:\n", buf); assertString(" columns:\n", buf);
ParserTernaryOperatorExpression expr_parser; ParserExpressionWithOptionalAlias expr_parser(false);
const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); const DataTypeFactory & data_type_factory = DataTypeFactory::instance();
ColumnsDescription<true> result{}; ColumnsDescription<true> result{};
...@@ -102,13 +103,9 @@ ColumnsDescription<true> ColumnsDescription<true>::parse(const String & str) ...@@ -102,13 +103,9 @@ ColumnsDescription<true> ColumnsDescription<true>::parse(const String & str)
readText(default_expr_str, buf); readText(default_expr_str, buf);
assertChar('\n', buf); assertChar('\n', buf);
ASTPtr default_expr;
Expected expected{};
const char * begin = default_expr_str.data(); const char * begin = default_expr_str.data();
const auto end = begin + default_expr_str.size(); const auto end = begin + default_expr_str.size();
const char * max_parsed_pos = begin; ASTPtr default_expr = parseQuery(expr_parser, begin, end, "default expression");
if (!expr_parser.parse(begin, end, default_expr, max_parsed_pos, expected))
throw Exception{"Could not parse default expression", DB::ErrorCodes::CANNOT_PARSE_TEXT};
if (ColumnDefaultType::Default == default_type) if (ColumnDefaultType::Default == default_type)
result.columns.emplace_back(column_name, std::move(type)); result.columns.emplace_back(column_name, std::move(type));
......
Markdown is supported
0% .
You are about to add 0 people to the discussion. Proceed with caution.
先完成此消息的编辑!
想要评论请 注册