diff --git a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h index 26e3abf9b559925e545c6f6f1341317e5d83fd59..41c1a42826ca21286f90218e3a027b79aa762e0f 100644 --- a/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h +++ b/dbms/src/AggregateFunctions/AggregateFunctionSequenceMatch.h @@ -4,11 +4,7 @@ #include #include #include -#include #include -#include -#include -#include #include #include #include @@ -164,23 +160,20 @@ public: if (!sufficientArgs(arg_count)) throw Exception{ "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) throw Exception{ "Aggregate function " + getName() + " supports up to " + 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(); if (!typeid_cast(time_arg)) throw Exception{ "Illegal type " + time_arg->getName() + " of first argument of aggregate function " + getName() + ", must be DateTime", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; for (const auto i : ext::range(1, arg_count)) { @@ -189,8 +182,7 @@ public: throw Exception{ "Illegal type " + cond_arg->getName() + " of argument " + toString(i + 1) + " of aggregate function " + getName() + ", must be UInt8", - ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT - }; + ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT}; } parsePattern(); @@ -275,52 +267,51 @@ private: actions.clear(); 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 auto begin = pos; - const auto end = pos + pattern.size(); - - ASTPtr node; - decltype(pos) max_parsed_pos{}; - Expected expected; + const char * begin = pos; + const char * end = pos + pattern.size(); - const auto throw_exception = [&] (const std::string & msg) + auto throw_exception = [&](const std::string & msg) { throw Exception{ msg + " '" + std::string(pos, end) + "' at position " + std::to_string(pos - begin), 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) { - if (special_open_p.ignore(pos, end)) + if (match("(?")) { - if (t_p.ignore(pos, end)) + if (match("t")) { PatternActionType type; - if (less_or_equal_p.ignore(pos, end)) + if (match("<=")) type = PatternActionType::TimeLessOrEqual; - else if (less_p.ignore(pos, end)) + else if (match("<")) type = PatternActionType::TimeLess; - else if (greater_or_equal_p.ignore(pos, end)) + else if (match(">=")) type = PatternActionType::TimeGreaterOrEqual; - else if (greater_p.ignore(pos, end)) + else if (match(">")) type = PatternActionType::TimeGreater; else 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"); if (actions.back().type != PatternActionType::SpecificEvent && @@ -328,32 +319,33 @@ private: actions.back().type != PatternActionType::KleeneStar) throw Exception{ "Temporal condition should be preceeded by an event condition", - ErrorCodes::BAD_ARGUMENTS - }; + ErrorCodes::BAD_ARGUMENTS}; - actions.emplace_back(type, typeid_cast(*node).value.safeGet()); + actions.emplace_back(type, duration); } - else if (number_p.parse(pos, node, expected)) + else { - const auto event_number = typeid_cast(*node).value.safeGet(); + 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) throw Exception{ "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); } - else - throw_exception("Unexpected special sequence"); - if (!special_close_p.ignore(pos, end)) + if (!match(")")) throw_exception("Expected closing parenthesis, found"); } - else if (dot_closure_p.ignore(pos, end)) + else if (match(".*")) actions.emplace_back(PatternActionType::KleeneStar); - else if (dot_p.ignore(pos, end)) + else if (match(".")) actions.emplace_back(PatternActionType::AnyEvent); else throw_exception("Could not parse pattern, unexpected starting symbol"); diff --git a/dbms/src/DataStreams/ValuesRowInputStream.cpp b/dbms/src/DataStreams/ValuesRowInputStream.cpp index b39b9abf53390534b7a54946cde180d0ad8edd73..3ccecb0c0274cc0a163913b399ed4d89e87a144b 100644 --- a/dbms/src/DataStreams/ValuesRowInputStream.cpp +++ b/dbms/src/DataStreams/ValuesRowInputStream.cpp @@ -1,6 +1,7 @@ #include #include #include +#include #include #include #include @@ -85,7 +86,7 @@ bool ValuesRowInputStream::read(Block & block) || e.code() == ErrorCodes::CANNOT_PARSE_DATETIME || 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 (istr.count() - istr.offset() != prev_istr_bytes) @@ -96,18 +97,18 @@ bool ValuesRowInputStream::read(Block & block) IDataType & type = *block.safeGetByPosition(i).type; - IParser::Pos pos = prev_istr_position; - Expected expected = ""; - IParser::Pos max_parsed_pos = pos; + + Tokens tokens(prev_istr_position, istr.buffer().end()); + TokenIterator token_iterator(tokens); 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: " + String(prev_istr_position, std::min(SHOW_CHARS_ON_SYNTAX_ERROR, istr.buffer().end() - prev_istr_position)), ErrorCodes::SYNTAX_ERROR); - istr.position() = const_cast(max_parsed_pos); + istr.position() = const_cast(token_iterator->begin); std::pair value_raw = evaluateConstantExpression(ast, context); Field value = convertFieldToType(value_raw.first, type, value_raw.second.get()); diff --git a/dbms/src/DataTypes/DataTypeFactory.cpp b/dbms/src/DataTypes/DataTypeFactory.cpp index 6b074b98c1d943f685b01f6afda45335c589512f..5066fdfb5effffe09a1adb0e83233a5e27c0c187 100644 --- a/dbms/src/DataTypes/DataTypeFactory.cpp +++ b/dbms/src/DataTypes/DataTypeFactory.cpp @@ -71,7 +71,7 @@ DataTypeFactory::DataTypeFactory() template inline DataTypePtr parseEnum(const String & name, const String & base_name, const String & parameters) { - ParserList parser{std::make_unique(), std::make_unique(","), false}; + ParserList parser{std::make_unique(), std::make_unique(TokenType::Comma), false}; ASTPtr elements = parseQuery(parser, parameters.data(), parameters.data() + parameters.size(), "parameters for enum type " + name); diff --git a/dbms/src/Interpreters/DDLWorker.cpp b/dbms/src/Interpreters/DDLWorker.cpp index 6385809fd4d7fc740266b00a9124f49a92e5ddcb..cb457d3b6597c9409f6d05dd98cdaa5bf7a15a7c 100644 --- a/dbms/src/Interpreters/DDLWorker.cpp +++ b/dbms/src/Interpreters/DDLWorker.cpp @@ -263,9 +263,9 @@ void DDLWorker::processTask(const DDLLogEntry & node, const std::string & node_n { ASTPtr query_ast; { - ParserQuery parser_query; - String description; 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); } diff --git a/dbms/src/Interpreters/executeQuery.cpp b/dbms/src/Interpreters/executeQuery.cpp index 14ef54ad2f229912296514ce5694a52c40059d59..d9a358dc13ef94ad73bab359ce660778310f6149 100644 --- a/dbms/src/Interpreters/executeQuery.cpp +++ b/dbms/src/Interpreters/executeQuery.cpp @@ -131,7 +131,7 @@ static std::tuple executeQueryImpl( const Settings & settings = context.getSettingsRef(); - ParserQuery parser; + ParserQuery parser(end); ASTPtr ast; size_t query_size; size_t max_query_size = settings.max_query_size; diff --git a/dbms/src/Parsers/ExpressionListParsers.cpp b/dbms/src/Parsers/ExpressionListParsers.cpp index dbf27c0d8ca20ce6ea202663e7b54338a52fceef..83ae6354065d117af13884671938dfc7ec8caa0c 100644 --- a/dbms/src/Parsers/ExpressionListParsers.cpp +++ b/dbms/src/Parsers/ExpressionListParsers.cpp @@ -533,13 +533,13 @@ bool ParserTupleElementExpression::parseImpl(Pos & pos, ASTPtr & node, Expected ParserExpressionWithOptionalAlias::ParserExpressionWithOptionalAlias(bool allow_alias_without_as_keyword) - : impl(std::make_unique(std::make_unique(), allow_alias_without_as_keyword)) + : impl(std::make_unique(std::make_unique(), allow_alias_without_as_keyword)) { } ParserExpressionInCastExpression::ParserExpressionInCastExpression(bool allow_alias_without_as_keyword) - : impl(std::make_unique(std::make_unique(), allow_alias_without_as_keyword)) + : impl(std::make_unique(std::make_unique(), allow_alias_without_as_keyword)) { } diff --git a/dbms/src/Parsers/ExpressionListParsers.h b/dbms/src/Parsers/ExpressionListParsers.h index 0a7833a9381e7856620b2b1b04cdde285497f604..73ac7dd63af4318ab41ac9ded44a73bc447cf434 100644 --- a/dbms/src/Parsers/ExpressionListParsers.h +++ b/dbms/src/Parsers/ExpressionListParsers.h @@ -305,6 +305,9 @@ protected: }; +using ParserExpression = ParserLambdaExpression; + + class ParserExpressionWithOptionalAlias : public IParserBase { public: diff --git a/dbms/src/Parsers/ParserAlterQuery.cpp b/dbms/src/Parsers/ParserAlterQuery.cpp index d86c5103fceff5959eda36c8c9cedfce73bc8c3d..033e430249c74f64e847c3e001784a29e9a819c5 100644 --- a/dbms/src/Parsers/ParserAlterQuery.cpp +++ b/dbms/src/Parsers/ParserAlterQuery.cpp @@ -217,7 +217,7 @@ bool ParserAlterQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) else if (s_reshard.ignore(pos, expected)) { ParserList weighted_zookeeper_paths_p(std::make_unique(), std::make_unique(TokenType::Comma), false); - ParserExpressionWithOptionalAlias parser_sharding_key_expr(false); + ParserExpression parser_sharding_key_expr(false); ParserStringLiteral parser_coordinator; if (s_copy.ignore(pos, expected)) diff --git a/dbms/src/Parsers/ParserCase.cpp b/dbms/src/Parsers/ParserCase.cpp index 82cf144416b4499b85680b0f89e31b2d4a946177..c740f7a4789981d68cc6fe377a79240fcac1c5e4 100644 --- a/dbms/src/Parsers/ParserCase.cpp +++ b/dbms/src/Parsers/ParserCase.cpp @@ -17,7 +17,7 @@ bool ParserCase::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserKeyword s_then{"THEN"}; ParserKeyword s_else{"ELSE"}; ParserKeyword s_end{ "END"}; - ParserExpressionWithOptionalAlias p_expr{false}; + ParserExpression p_expr{false}; if (!s_case.parse(pos, node, expected)) { diff --git a/dbms/src/Parsers/ParserKillQueryQuery.cpp b/dbms/src/Parsers/ParserKillQueryQuery.cpp index eb5b42a710e916ba95297237a44862da667c249d..46c8398573ba8e60342b7e36e2cc20f3b9d37070 100644 --- a/dbms/src/Parsers/ParserKillQueryQuery.cpp +++ b/dbms/src/Parsers/ParserKillQueryQuery.cpp @@ -27,7 +27,7 @@ bool ParserKillQueryQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expect if (!ParserKeyword{"WHERE"}.ignore(pos, expected)) return false; - ParserExpressionWithOptionalAlias p_where_expression(false); + ParserExpression p_where_expression(false); if (!p_where_expression.parse(pos, query->where_expression, expected)) return false; diff --git a/dbms/src/Parsers/ParserSelectQuery.cpp b/dbms/src/Parsers/ParserSelectQuery.cpp index 00acc6cd11ac4419a508d97518389245917917ef..1ddd63fd91cbcb8bcc1f7405650ff09e62c70d56 100644 --- a/dbms/src/Parsers/ParserSelectQuery.cpp +++ b/dbms/src/Parsers/ParserSelectQuery.cpp @@ -44,7 +44,7 @@ bool ParserSelectQuery::parseImpl(Pos & pos, ASTPtr & node, Expected & expected) ParserNotEmptyExpressionList exp_list(false); ParserNotEmptyExpressionList exp_list_for_select_clause(true); /// Allows aliases without AS keyword. - ParserExpressionWithOptionalAlias exp_elem(false); + ParserExpression exp_elem(false); ParserOrderByExpressionList order_list; /// SELECT [DISTINCT] expr list diff --git a/dbms/src/Storages/ColumnsDescription.cpp b/dbms/src/Storages/ColumnsDescription.cpp index 42df42c4278e6bf24517f5446412afe2f6784407..ec99378c17e3bccf39e9ff1229ab98be050d5ec9 100644 --- a/dbms/src/Storages/ColumnsDescription.cpp +++ b/dbms/src/Storages/ColumnsDescription.cpp @@ -1,4 +1,5 @@ #include +#include #include #include #include @@ -71,7 +72,7 @@ ColumnsDescription ColumnsDescription::parse(const String & str) readText(count, buf); assertString(" columns:\n", buf); - ParserTernaryOperatorExpression expr_parser; + ParserExpressionWithOptionalAlias expr_parser(false); const DataTypeFactory & data_type_factory = DataTypeFactory::instance(); ColumnsDescription result{}; @@ -102,13 +103,9 @@ ColumnsDescription ColumnsDescription::parse(const String & str) readText(default_expr_str, buf); assertChar('\n', buf); - ASTPtr default_expr; - Expected expected{}; const char * begin = default_expr_str.data(); const auto end = begin + default_expr_str.size(); - const char * max_parsed_pos = begin; - if (!expr_parser.parse(begin, end, default_expr, max_parsed_pos, expected)) - throw Exception{"Could not parse default expression", DB::ErrorCodes::CANNOT_PARSE_TEXT}; + ASTPtr default_expr = parseQuery(expr_parser, begin, end, "default expression"); if (ColumnDefaultType::Default == default_type) result.columns.emplace_back(column_name, std::move(type));