This commit is contained in:
Alexey Arno 2015-04-14 17:21:19 +03:00
commit 9bcb33c1d7
18 changed files with 60 additions and 122 deletions

View File

@ -1,7 +1,6 @@
#pragma once
#include <DB/Core/QueryProcessingStage.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Interpreters/InterpreterQuery.h>

View File

@ -1,12 +1,7 @@
#pragma once
#include <list>
#include <Poco/SharedPtr.h>
#include <DB/Core/Types.h>
#include <DB/Parsers/IParser.h>
#include <iostream>
namespace DB
{
@ -16,27 +11,7 @@ namespace DB
class IParserBase : public IParser
{
public:
bool parse(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos new_max_parsed_pos = pos;
Expected new_expected = getName();
bool res = parseImpl(pos, end, node, new_max_parsed_pos, new_expected);
if (new_max_parsed_pos > max_parsed_pos)
max_parsed_pos = new_max_parsed_pos;
if (new_max_parsed_pos >= max_parsed_pos)
expected = new_expected;
if (!res)
node = nullptr;
if (pos > end)
throw Exception("Logical error: pos > end.", ErrorCodes::LOGICAL_ERROR);
return res;
}
bool parse(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected);
protected:
virtual bool parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected) = 0;

View File

@ -83,7 +83,6 @@ bool IParserNameTypePair<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr & nod
return true;
}
pos = begin;
return false;
}
@ -119,10 +118,6 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr
ParserTernaryOperatorExpression expr_parser;
const auto begin = pos;
const auto reset_pos_and_return = [&pos, begin] {
pos = begin;
return false;
};
/// mandatory column name
ASTPtr name;
@ -160,10 +155,10 @@ bool IParserColumnDeclaration<NameParser>::parseImpl(Pos & pos, Pos end, ASTPtr
ws.ignore(pos, end, max_parsed_pos, expected);
if (!expr_parser.parse(pos, end, default_expression, max_parsed_pos, expected))
return reset_pos_and_return();
return false;
}
else if (!type)
return reset_pos_and_return(); /// reject sole column name without type
return false; /// reject sole column name without type
const auto column_declaration = new ASTColumnDeclaration{StringRange{begin, pos}};
node = column_declaration;

View File

@ -33,9 +33,6 @@
#include <DB/DataStreams/RemoteBlockInputStream.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Interpreters/Context.h>
#include <DB/Client/Connection.h>

View File

@ -18,6 +18,7 @@
#include <DB/Parsers/ASTExpressionList.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/parseQuery.h>
namespace DB
{
@ -69,15 +70,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const
Array params_row;
ParserExpressionList args_parser;
ASTPtr args_ast;
Expected expected = "";
IParser::Pos pos = parameters.data();
IParser::Pos end = pos + parameters.size();
IParser::Pos max_parsed_pos = pos;
if (!(args_parser.parse(pos, end, args_ast, max_parsed_pos, expected) && pos == end))
throw Exception("Cannot parse parameters for data type " + name, ErrorCodes::SYNTAX_ERROR);
ASTPtr args_ast = parseQuery(args_parser, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
ASTExpressionList & args_list = typeid_cast<ASTExpressionList &>(*args_ast);
if (args_list.children.empty())
@ -121,14 +114,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const
if (base_name == "Nested")
{
ParserNameTypePairList columns_p;
ASTPtr columns_ast;
Expected expected = "";
IParser::Pos pos = parameters.data();
IParser::Pos end = pos + parameters.size();
IParser::Pos max_parsed_pos = pos;
if (!(columns_p.parse(pos, end, columns_ast, max_parsed_pos, expected) && pos == end))
throw Exception("Cannot parse parameters for data type " + name, ErrorCodes::SYNTAX_ERROR);
ASTPtr columns_ast = parseQuery(columns_p, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
NamesAndTypesListPtr columns = new NamesAndTypesList;
@ -151,14 +137,7 @@ DataTypePtr DataTypeFactory::get(const String & name) const
if (base_name == "Tuple")
{
ParserExpressionList columns_p;
ASTPtr columns_ast;
Expected expected = "";
IParser::Pos pos = parameters.data();
IParser::Pos end = pos + parameters.size();
IParser::Pos max_parsed_pos = pos;
if (!(columns_p.parse(pos, end, columns_ast, max_parsed_pos, expected) && pos == end))
throw Exception("Cannot parse parameters for data type " + name, ErrorCodes::SYNTAX_ERROR);
ASTPtr columns_ast = parseQuery(columns_p, parameters.data(), parameters.data() + parameters.size(), "parameters for data type " + name);
DataTypes elems;

View File

@ -9,7 +9,6 @@
#include <DB/Parsers/ASTSubquery.h>
#include <DB/Parsers/ASTSet.h>
#include <DB/Parsers/ASTOrderByElement.h>
#include <DB/Parsers/ParserSelectQuery.h>
#include <DB/DataTypes/DataTypeSet.h>
#include <DB/DataTypes/DataTypeTuple.h>

View File

@ -17,8 +17,10 @@
#include <DB/Storages/StorageLog.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/Parsers/parseQuery.h>
#include <DB/Interpreters/InterpreterSelectQuery.h>
#include <DB/Interpreters/InterpreterCreateQuery.h>
@ -409,11 +411,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(const NamesAndTypesList & columns)
const auto end = pos + type_name->size();
ParserIdentifierWithOptionalParameters storage_p;
Expected expected{""};
IParser::Pos max_parsed_pos = pos;
if (!storage_p.parse(pos, end, column_declaration->type, max_parsed_pos, expected))
throw Exception("Cannot parse data type.", ErrorCodes::SYNTAX_ERROR);
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type->query_string = type_name;
columns_list.children.push_back(column_declaration_ptr);
}
@ -444,11 +442,7 @@ ASTPtr InterpreterCreateQuery::formatColumns(NamesAndTypesList columns,
const auto end = pos + type_name->size();
ParserIdentifierWithOptionalParameters storage_p;
Expected expected{""};
IParser::Pos max_parsed_pos = pos;
if (!storage_p.parse(pos, end, column_declaration->type, max_parsed_pos, expected))
throw Exception("Cannot parse data type.", ErrorCodes::SYNTAX_ERROR);
column_declaration->type = parseQuery(storage_p, pos, end, "data type");
column_declaration->type->query_string = type_name;
const auto it = column_defaults.find(column.name);

View File

@ -5,6 +5,7 @@
#include <DB/DataStreams/BlockIO.h>
#include <DB/Parsers/ASTInsertQuery.h>
#include <DB/Parsers/ASTShowProcesslistQuery.h>
#include <DB/Parsers/ParserQuery.h>
#include <DB/Parsers/parseQuery.h>
#include <DB/Interpreters/executeQuery.h>

View File

@ -227,15 +227,12 @@ TestResult check(const TestEntry & entry)
}
}
bool parse(DB::ASTPtr & ast, const std::string & query)
bool parse(DB::ASTPtr & ast, const std::string & query)
{
DB::ParserSelectQuery parser;
const char * pos = &query[0];
const char * end = &query[0] + query.size();
const char * max_parsed_pos = pos;
DB::Expected expected = "";
return parser.parse(pos, end, ast, max_parsed_pos, expected);
std::string message;
ast = DB::tryParseQuery(parser, query.data(), query.data() + query.size(), message, false, "");
return !ast.isNull();
}
bool equals(const DB::ASTPtr & lhs, const DB::ASTPtr & rhs)

View File

@ -407,23 +407,18 @@ bool ParserStringLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max
bool ParserLiteral::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
ParserNull null_p;
ParserNumber num_p;
ParserStringLiteral str_p;
if (null_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
if (num_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
if (str_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
expected = "literal: one of nullptr, number, single quoted string";
return false;
@ -462,34 +457,27 @@ bool ParserExpressionElement::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
if (subquery_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
if (paren_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
if (array_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
if (lit_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
if (fun_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
if (id_p.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
if (asterisk_p.parse(pos, end, node, max_parsed_pos, expected))
{
node = new ASTAsterisk(StringRange(begin, pos));
return true;
}
pos = begin;
expected = "expression element: one of array, literal, function, identifier, asterisk, parenthised expression, subquery";
return false;

View File

@ -330,10 +330,7 @@ bool ParserLambdaExpression::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos &
ws.ignore(pos, end, max_parsed_pos, expected);
if (!elem_parser.parse(pos, end, expression, max_parsed_pos, expected))
{
pos = begin;
return false;
}
/// lambda(tuple(inner_arguments), expression)

View File

@ -0,0 +1,36 @@
#include <DB/Parsers/IParserBase.h>
namespace DB
{
bool IParserBase::parse(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
Pos new_max_parsed_pos = pos;
Expected new_expected = getName();
bool res = parseImpl(pos, end, node, new_max_parsed_pos, new_expected);
if (pos > new_max_parsed_pos)
new_max_parsed_pos = pos;
if (new_max_parsed_pos > max_parsed_pos)
max_parsed_pos = new_max_parsed_pos;
if (new_max_parsed_pos >= max_parsed_pos)
expected = new_expected;
if (pos > end)
throw Exception("Logical error: pos > end.", ErrorCodes::LOGICAL_ERROR);
if (!res)
{
node = nullptr;
pos = begin;
}
return res;
}
}

View File

@ -54,20 +54,14 @@ bool ParserNestedTable::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_p
bool ParserIdentifierWithParameters::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_pos, Expected & expected)
{
Pos begin = pos;
ParserFunction function_or_array;
if (function_or_array.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
ParserNestedTable nested;
if (nested.parse(pos, end, node, max_parsed_pos, expected))
return true;
pos = begin;
return false;
}
@ -80,10 +74,7 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, Pos end, ASTPt
Pos begin = pos;
if (parametric.parse(pos, end, node, max_parsed_pos, expected))
{
return true;
}
pos = begin;
ASTPtr ident;
if (non_parametric.parse(pos, end, ident, max_parsed_pos, expected))
@ -93,7 +84,6 @@ bool ParserIdentifierWithOptionalParameters::parseImpl(Pos & pos, Pos end, ASTPt
func->name = typeid_cast<ASTIdentifier &>(*ident).name;
return true;
}
pos = begin;
return false;
}

View File

@ -52,7 +52,7 @@ bool ParserQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & max_parsed_
|| check_p.parse(pos, end, node, max_parsed_pos, expected);
/* || multi_p.parse(pos, end, node, max_parsed_pos, expected)*/;
if (!res)
if (!res && (!expected || !*expected))
expected = "One of: SHOW TABLES, SHOW DATABASES, SHOW CREATE TABLE, SELECT, INSERT, CREATE, ATTACH, RENAME, DROP, DETACH, USE, SET, OPTIMIZE, EXISTS, DESCRIBE, DESC, ALTER, SHOW PROCESSLIST, CHECK, opening curly brace";
return res;

View File

@ -76,10 +76,7 @@ bool ParserShowTablesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Pos & m
return false;
}
else
{
pos = begin;
return false;
}
ws.ignore(pos, end);

View File

@ -44,10 +44,7 @@ bool ParserTablePropertiesQuery::parseImpl(Pos & pos, Pos end, ASTPtr & node, Po
ws.ignore(pos, end);
if (!s_create.ignore(pos, end, max_parsed_pos, expected))
{
pos = begin;
return false;
}
query_ptr = new ASTShowCreateQuery;
}

View File

@ -40,7 +40,10 @@ static std::string getSyntaxErrorMessage(
if (max_parsed_pos == end || *max_parsed_pos == ';')
{
message << ": failed at end of query";
message << ": failed at end of query.\n";
if (expected && *expected && *expected != '.')
message << "Expected " << expected;
}
else
{

View File

@ -1,19 +1,13 @@
#include <DB/Storages/StorageChunkMerger.h>
#include <DB/Storages/StorageChunks.h>
#include <DB/Storages/StorageChunkRef.h>
#include <DB/Parsers/ASTCreateQuery.h>
#include <DB/Parsers/ASTNameTypePair.h>
#include <DB/Parsers/ASTLiteral.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTDropQuery.h>
#include <DB/Parsers/ParserCreateQuery.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Interpreters/executeQuery.h>
#include <DB/Interpreters/InterpreterDropQuery.h>
#include <DB/DataStreams/ConcatBlockInputStream.h>
#include <DB/DataStreams/AddingDefaultBlockInputStream.h>
#include <DB/DataStreams/AddingConstColumnBlockInputStream.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Common/VirtualColumnUtils.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/Columns/ColumnString.h>